From 461de6e6e9654913cb6a4b0c183f727c1f41fe5a Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Fri, 9 Dec 2022 16:16:01 -0500 Subject: [PATCH] Revert "Ensure PartitionedOutputOperator is run with fixed local distribution" This reverts commit 1c062024c494baa1d12940110c3e47618a245f33. --- .../io/trino/SystemSessionProperties.java | 11 - .../io/trino/sql/planner/OptimizerConfig.java | 14 - ...wPartialAggregationOverGroupIdRuleSet.java | 28 +- .../optimizations/AddLocalExchanges.java | 4 - .../trino/sql/planner/plan/ExchangeNode.java | 8 - .../io/trino/cost/TestOptimizerConfig.java | 7 +- .../planner/TestAddDynamicFilterSource.java | 112 ++---- .../java/io/trino/sql/planner/TestInsert.java | 6 +- .../trino/sql/planner/TestLogicalPlanner.java | 48 ++- .../TestTableScanNodePartitioning.java | 7 +- .../optimizations/TestAddExchangesPlans.java | 46 ++- ...tAddLocalExchangesForTaskScaleWriters.java | 27 +- .../TestFullOuterJoinWithCoalesce.java | 62 ++-- .../sql/planner/optimizations/TestWindow.java | 55 ++- .../plugin/jdbc/BaseJdbcConnectorTest.java | 11 +- .../plugin/hive/BaseHiveConnectorTest.java | 2 +- .../plugin/hive/optimizer/TestHivePlans.java | 84 ++--- .../mysql/TestMySqlLegacyConnectorTest.java | 2 +- ...asePinotIntegrationConnectorSmokeTest.java | 44 +-- ...aultTolerantExecutionDynamicFiltering.java | 5 +- .../tpcds/hive/partitioned/q01.plan.txt | 35 +- .../tpcds/hive/partitioned/q02.plan.txt | 67 ++-- .../tpcds/hive/partitioned/q03.plan.txt | 15 +- .../tpcds/hive/partitioned/q04.plan.txt | 180 +++++----- .../tpcds/hive/partitioned/q05.plan.txt | 6 +- .../tpcds/hive/partitioned/q06.plan.txt | 46 ++- .../tpcds/hive/partitioned/q07.plan.txt | 18 +- .../tpcds/hive/partitioned/q08.plan.txt | 43 +-- .../tpcds/hive/partitioned/q10.plan.txt | 86 +++-- .../tpcds/hive/partitioned/q11.plan.txt | 112 +++--- .../tpcds/hive/partitioned/q12.plan.txt | 31 +- .../tpcds/hive/partitioned/q13.plan.txt | 18 +- .../tpcds/hive/partitioned/q14.plan.txt | 278 +++++++-------- .../tpcds/hive/partitioned/q15.plan.txt | 17 +- .../tpcds/hive/partitioned/q16.plan.txt | 55 ++- .../tpcds/hive/partitioned/q17.plan.txt | 59 ++-- .../tpcds/hive/partitioned/q18.plan.txt | 53 ++- .../tpcds/hive/partitioned/q19.plan.txt | 33 +- .../tpcds/hive/partitioned/q20.plan.txt | 31 +- .../tpcds/hive/partitioned/q21.plan.txt | 17 +- .../tpcds/hive/partitioned/q23.plan.txt | 140 ++++---- .../tpcds/hive/partitioned/q24.plan.txt | 83 ++--- .../tpcds/hive/partitioned/q25.plan.txt | 67 ++-- .../tpcds/hive/partitioned/q26.plan.txt | 18 +- .../tpcds/hive/partitioned/q27.plan.txt | 18 +- .../tpcds/hive/partitioned/q28.plan.txt | 18 +- .../tpcds/hive/partitioned/q29.plan.txt | 65 ++-- .../tpcds/hive/partitioned/q30.plan.txt | 75 ++-- .../tpcds/hive/partitioned/q31.plan.txt | 145 ++++---- .../tpcds/hive/partitioned/q32.plan.txt | 24 +- .../tpcds/hive/partitioned/q33.plan.txt | 100 +++--- .../tpcds/hive/partitioned/q34.plan.txt | 17 +- .../tpcds/hive/partitioned/q35.plan.txt | 48 ++- .../tpcds/hive/partitioned/q36.plan.txt | 16 +- .../tpcds/hive/partitioned/q37.plan.txt | 23 +- .../tpcds/hive/partitioned/q38.plan.txt | 42 +-- .../tpcds/hive/partitioned/q39.plan.txt | 60 ++-- .../tpcds/hive/partitioned/q40.plan.txt | 6 +- .../tpcds/hive/partitioned/q41.plan.txt | 15 +- .../tpcds/hive/partitioned/q42.plan.txt | 15 +- .../tpcds/hive/partitioned/q43.plan.txt | 30 +- .../tpcds/hive/partitioned/q44.plan.txt | 60 ++-- .../tpcds/hive/partitioned/q45.plan.txt | 17 +- .../tpcds/hive/partitioned/q46.plan.txt | 50 ++- .../tpcds/hive/partitioned/q47.plan.txt | 90 +++-- .../tpcds/hive/partitioned/q48.plan.txt | 16 +- .../tpcds/hive/partitioned/q49.plan.txt | 102 +++--- .../tpcds/hive/partitioned/q50.plan.txt | 25 +- .../tpcds/hive/partitioned/q51.plan.txt | 51 ++- .../tpcds/hive/partitioned/q52.plan.txt | 15 +- .../tpcds/hive/partitioned/q53.plan.txt | 39 +-- .../tpcds/hive/partitioned/q54.plan.txt | 117 +++---- .../tpcds/hive/partitioned/q55.plan.txt | 15 +- .../tpcds/hive/partitioned/q56.plan.txt | 99 +++--- .../tpcds/hive/partitioned/q57.plan.txt | 90 +++-- .../tpcds/hive/partitioned/q58.plan.txt | 126 ++++--- .../tpcds/hive/partitioned/q59.plan.txt | 74 ++-- .../tpcds/hive/partitioned/q60.plan.txt | 99 +++--- .../tpcds/hive/partitioned/q61.plan.txt | 54 ++- .../tpcds/hive/partitioned/q62.plan.txt | 19 +- .../tpcds/hive/partitioned/q63.plan.txt | 39 +-- .../tpcds/hive/partitioned/q64.plan.txt | 304 ++++++++-------- .../tpcds/hive/partitioned/q65.plan.txt | 38 +- .../tpcds/hive/partitioned/q66.plan.txt | 68 ++-- .../tpcds/hive/partitioned/q68.plan.txt | 49 ++- .../tpcds/hive/partitioned/q69.plan.txt | 60 ++-- .../tpcds/hive/partitioned/q70.plan.txt | 28 +- .../tpcds/hive/partitioned/q71.plan.txt | 3 +- .../tpcds/hive/partitioned/q72.plan.txt | 70 ++-- .../tpcds/hive/partitioned/q73.plan.txt | 17 +- .../tpcds/hive/partitioned/q74.plan.txt | 116 +++---- .../tpcds/hive/partitioned/q75.plan.txt | 198 +++++------ .../tpcds/hive/partitioned/q76.plan.txt | 44 ++- .../tpcds/hive/partitioned/q77.plan.txt | 86 +++-- .../tpcds/hive/partitioned/q78.plan.txt | 18 +- .../tpcds/hive/partitioned/q79.plan.txt | 33 +- .../tpcds/hive/partitioned/q80.plan.txt | 18 +- .../tpcds/hive/partitioned/q81.plan.txt | 75 ++-- .../tpcds/hive/partitioned/q82.plan.txt | 23 +- .../tpcds/hive/partitioned/q83.plan.txt | 144 ++++---- .../tpcds/hive/partitioned/q85.plan.txt | 63 ++-- .../tpcds/hive/partitioned/q86.plan.txt | 15 +- .../tpcds/hive/partitioned/q87.plan.txt | 42 +-- .../tpcds/hive/partitioned/q89.plan.txt | 30 +- .../tpcds/hive/partitioned/q91.plan.txt | 25 +- .../tpcds/hive/partitioned/q92.plan.txt | 24 +- .../tpcds/hive/partitioned/q93.plan.txt | 6 +- .../tpcds/hive/partitioned/q94.plan.txt | 23 +- .../tpcds/hive/partitioned/q95.plan.txt | 62 ++-- .../tpcds/hive/partitioned/q97.plan.txt | 26 +- .../tpcds/hive/partitioned/q98.plan.txt | 31 +- .../tpcds/hive/partitioned/q99.plan.txt | 19 +- .../tpcds/hive/unpartitioned/q01.plan.txt | 51 ++- .../tpcds/hive/unpartitioned/q02.plan.txt | 67 ++-- .../tpcds/hive/unpartitioned/q03.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q04.plan.txt | 180 +++++----- .../tpcds/hive/unpartitioned/q05.plan.txt | 6 +- .../tpcds/hive/unpartitioned/q06.plan.txt | 43 +-- .../tpcds/hive/unpartitioned/q07.plan.txt | 19 +- .../tpcds/hive/unpartitioned/q08.plan.txt | 43 +-- .../tpcds/hive/unpartitioned/q10.plan.txt | 84 +++-- .../tpcds/hive/unpartitioned/q11.plan.txt | 120 +++---- .../tpcds/hive/unpartitioned/q12.plan.txt | 28 +- .../tpcds/hive/unpartitioned/q14.plan.txt | 324 +++++++++--------- .../tpcds/hive/unpartitioned/q15.plan.txt | 17 +- .../tpcds/hive/unpartitioned/q16.plan.txt | 43 ++- .../tpcds/hive/unpartitioned/q17.plan.txt | 52 ++- .../tpcds/hive/unpartitioned/q18.plan.txt | 9 +- .../tpcds/hive/unpartitioned/q19.plan.txt | 21 +- .../tpcds/hive/unpartitioned/q20.plan.txt | 28 +- .../tpcds/hive/unpartitioned/q21.plan.txt | 17 +- .../tpcds/hive/unpartitioned/q23.plan.txt | 140 ++++---- .../tpcds/hive/unpartitioned/q24.plan.txt | 81 ++--- .../tpcds/hive/unpartitioned/q25.plan.txt | 46 ++- .../tpcds/hive/unpartitioned/q26.plan.txt | 19 +- .../tpcds/hive/unpartitioned/q27.plan.txt | 18 +- .../tpcds/hive/unpartitioned/q28.plan.txt | 18 +- .../tpcds/hive/unpartitioned/q29.plan.txt | 46 ++- .../tpcds/hive/unpartitioned/q30.plan.txt | 75 ++-- .../tpcds/hive/unpartitioned/q31.plan.txt | 151 ++++---- .../tpcds/hive/unpartitioned/q32.plan.txt | 24 +- .../tpcds/hive/unpartitioned/q33.plan.txt | 102 +++--- .../tpcds/hive/unpartitioned/q34.plan.txt | 33 +- .../tpcds/hive/unpartitioned/q35.plan.txt | 46 ++- .../tpcds/hive/unpartitioned/q36.plan.txt | 17 +- .../tpcds/hive/unpartitioned/q37.plan.txt | 23 +- .../tpcds/hive/unpartitioned/q38.plan.txt | 42 +-- .../tpcds/hive/unpartitioned/q39.plan.txt | 60 ++-- .../tpcds/hive/unpartitioned/q40.plan.txt | 6 +- .../tpcds/hive/unpartitioned/q41.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q42.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q43.plan.txt | 30 +- .../tpcds/hive/unpartitioned/q44.plan.txt | 60 ++-- .../tpcds/hive/unpartitioned/q45.plan.txt | 17 +- .../tpcds/hive/unpartitioned/q46.plan.txt | 45 ++- .../tpcds/hive/unpartitioned/q47.plan.txt | 90 +++-- .../tpcds/hive/unpartitioned/q49.plan.txt | 111 +++--- .../tpcds/hive/unpartitioned/q50.plan.txt | 14 +- .../tpcds/hive/unpartitioned/q51.plan.txt | 51 ++- .../tpcds/hive/unpartitioned/q52.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q53.plan.txt | 30 +- .../tpcds/hive/unpartitioned/q54.plan.txt | 117 +++---- .../tpcds/hive/unpartitioned/q55.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q56.plan.txt | 100 +++--- .../tpcds/hive/unpartitioned/q57.plan.txt | 90 +++-- .../tpcds/hive/unpartitioned/q58.plan.txt | 126 ++++--- .../tpcds/hive/unpartitioned/q59.plan.txt | 63 ++-- .../tpcds/hive/unpartitioned/q60.plan.txt | 102 +++--- .../tpcds/hive/unpartitioned/q61.plan.txt | 52 ++- .../tpcds/hive/unpartitioned/q62.plan.txt | 19 +- .../tpcds/hive/unpartitioned/q63.plan.txt | 30 +- .../tpcds/hive/unpartitioned/q64.plan.txt | 310 ++++++++--------- .../tpcds/hive/unpartitioned/q65.plan.txt | 38 +- .../tpcds/hive/unpartitioned/q66.plan.txt | 68 ++-- .../tpcds/hive/unpartitioned/q68.plan.txt | 45 ++- .../tpcds/hive/unpartitioned/q69.plan.txt | 93 +++-- .../tpcds/hive/unpartitioned/q70.plan.txt | 28 +- .../tpcds/hive/unpartitioned/q72.plan.txt | 32 +- .../tpcds/hive/unpartitioned/q73.plan.txt | 33 +- .../tpcds/hive/unpartitioned/q74.plan.txt | 120 +++---- .../tpcds/hive/unpartitioned/q75.plan.txt | 198 +++++------ .../tpcds/hive/unpartitioned/q76.plan.txt | 39 +-- .../tpcds/hive/unpartitioned/q77.plan.txt | 86 +++-- .../tpcds/hive/unpartitioned/q78.plan.txt | 88 +++-- .../tpcds/hive/unpartitioned/q79.plan.txt | 33 +- .../tpcds/hive/unpartitioned/q80.plan.txt | 18 +- .../tpcds/hive/unpartitioned/q81.plan.txt | 47 ++- .../tpcds/hive/unpartitioned/q82.plan.txt | 23 +- .../tpcds/hive/unpartitioned/q83.plan.txt | 144 ++++---- .../tpcds/hive/unpartitioned/q85.plan.txt | 65 ++-- .../tpcds/hive/unpartitioned/q86.plan.txt | 15 +- .../tpcds/hive/unpartitioned/q87.plan.txt | 42 +-- .../tpcds/hive/unpartitioned/q89.plan.txt | 30 +- .../tpcds/hive/unpartitioned/q91.plan.txt | 25 +- .../tpcds/hive/unpartitioned/q92.plan.txt | 24 +- .../tpcds/hive/unpartitioned/q93.plan.txt | 6 +- .../tpcds/hive/unpartitioned/q94.plan.txt | 43 ++- .../tpcds/hive/unpartitioned/q95.plan.txt | 30 +- .../tpcds/hive/unpartitioned/q97.plan.txt | 26 +- .../tpcds/hive/unpartitioned/q98.plan.txt | 28 +- .../tpcds/hive/unpartitioned/q99.plan.txt | 19 +- .../iceberg/orc/partitioned/q01.plan.txt | 51 ++- .../iceberg/orc/partitioned/q02.plan.txt | 67 ++-- .../iceberg/orc/partitioned/q03.plan.txt | 15 +- .../iceberg/orc/partitioned/q04.plan.txt | 180 +++++----- .../iceberg/orc/partitioned/q05.plan.txt | 6 +- .../iceberg/orc/partitioned/q06.plan.txt | 43 +-- .../iceberg/orc/partitioned/q07.plan.txt | 19 +- .../iceberg/orc/partitioned/q08.plan.txt | 43 +-- .../iceberg/orc/partitioned/q10.plan.txt | 84 +++-- .../iceberg/orc/partitioned/q11.plan.txt | 120 +++---- .../iceberg/orc/partitioned/q12.plan.txt | 31 +- .../iceberg/orc/partitioned/q13.plan.txt | 17 +- .../iceberg/orc/partitioned/q14.plan.txt | 324 +++++++++--------- .../iceberg/orc/partitioned/q15.plan.txt | 17 +- .../iceberg/orc/partitioned/q16.plan.txt | 43 ++- .../iceberg/orc/partitioned/q17.plan.txt | 46 ++- .../iceberg/orc/partitioned/q18.plan.txt | 52 ++- .../iceberg/orc/partitioned/q19.plan.txt | 19 +- .../iceberg/orc/partitioned/q20.plan.txt | 28 +- .../iceberg/orc/partitioned/q21.plan.txt | 17 +- .../iceberg/orc/partitioned/q23.plan.txt | 140 ++++---- .../iceberg/orc/partitioned/q24.plan.txt | 75 ++-- .../iceberg/orc/partitioned/q25.plan.txt | 44 ++- .../iceberg/orc/partitioned/q26.plan.txt | 19 +- .../iceberg/orc/partitioned/q27.plan.txt | 28 +- .../iceberg/orc/partitioned/q28.plan.txt | 18 +- .../iceberg/orc/partitioned/q29.plan.txt | 44 ++- .../iceberg/orc/partitioned/q30.plan.txt | 75 ++-- .../iceberg/orc/partitioned/q31.plan.txt | 151 ++++---- .../iceberg/orc/partitioned/q32.plan.txt | 24 +- .../iceberg/orc/partitioned/q33.plan.txt | 102 +++--- .../iceberg/orc/partitioned/q34.plan.txt | 33 +- .../iceberg/orc/partitioned/q35.plan.txt | 46 ++- .../iceberg/orc/partitioned/q36.plan.txt | 17 +- .../iceberg/orc/partitioned/q37.plan.txt | 23 +- .../iceberg/orc/partitioned/q38.plan.txt | 42 +-- .../iceberg/orc/partitioned/q39.plan.txt | 60 ++-- .../iceberg/orc/partitioned/q40.plan.txt | 6 +- .../iceberg/orc/partitioned/q41.plan.txt | 15 +- .../iceberg/orc/partitioned/q42.plan.txt | 15 +- .../iceberg/orc/partitioned/q43.plan.txt | 30 +- .../iceberg/orc/partitioned/q44.plan.txt | 60 ++-- .../iceberg/orc/partitioned/q45.plan.txt | 17 +- .../iceberg/orc/partitioned/q46.plan.txt | 45 ++- .../iceberg/orc/partitioned/q47.plan.txt | 90 +++-- .../iceberg/orc/partitioned/q48.plan.txt | 17 +- .../iceberg/orc/partitioned/q49.plan.txt | 111 +++--- .../iceberg/orc/partitioned/q50.plan.txt | 14 +- .../iceberg/orc/partitioned/q51.plan.txt | 51 ++- .../iceberg/orc/partitioned/q52.plan.txt | 15 +- .../iceberg/orc/partitioned/q53.plan.txt | 30 +- .../iceberg/orc/partitioned/q54.plan.txt | 117 +++---- .../iceberg/orc/partitioned/q55.plan.txt | 15 +- .../iceberg/orc/partitioned/q56.plan.txt | 102 +++--- .../iceberg/orc/partitioned/q57.plan.txt | 90 +++-- .../iceberg/orc/partitioned/q58.plan.txt | 123 +++---- .../iceberg/orc/partitioned/q59.plan.txt | 63 ++-- .../iceberg/orc/partitioned/q60.plan.txt | 99 +++--- .../iceberg/orc/partitioned/q61.plan.txt | 52 ++- .../iceberg/orc/partitioned/q62.plan.txt | 19 +- .../iceberg/orc/partitioned/q63.plan.txt | 30 +- .../iceberg/orc/partitioned/q64.plan.txt | 318 ++++++++--------- .../iceberg/orc/partitioned/q65.plan.txt | 38 +- .../iceberg/orc/partitioned/q66.plan.txt | 68 ++-- .../iceberg/orc/partitioned/q68.plan.txt | 45 ++- .../iceberg/orc/partitioned/q69.plan.txt | 63 ++-- .../iceberg/orc/partitioned/q70.plan.txt | 28 +- .../iceberg/orc/partitioned/q72.plan.txt | 32 +- .../iceberg/orc/partitioned/q73.plan.txt | 33 +- .../iceberg/orc/partitioned/q74.plan.txt | 120 +++---- .../iceberg/orc/partitioned/q75.plan.txt | 198 +++++------ .../iceberg/orc/partitioned/q76.plan.txt | 38 +- .../iceberg/orc/partitioned/q77.plan.txt | 86 +++-- .../iceberg/orc/partitioned/q78.plan.txt | 88 +++-- .../iceberg/orc/partitioned/q79.plan.txt | 33 +- .../iceberg/orc/partitioned/q80.plan.txt | 18 +- .../iceberg/orc/partitioned/q81.plan.txt | 75 ++-- .../iceberg/orc/partitioned/q82.plan.txt | 23 +- .../iceberg/orc/partitioned/q83.plan.txt | 144 ++++---- .../iceberg/orc/partitioned/q85.plan.txt | 65 ++-- .../iceberg/orc/partitioned/q86.plan.txt | 15 +- .../iceberg/orc/partitioned/q87.plan.txt | 42 +-- .../iceberg/orc/partitioned/q89.plan.txt | 30 +- .../iceberg/orc/partitioned/q91.plan.txt | 28 +- .../iceberg/orc/partitioned/q92.plan.txt | 24 +- .../iceberg/orc/partitioned/q93.plan.txt | 6 +- .../iceberg/orc/partitioned/q94.plan.txt | 43 ++- .../iceberg/orc/partitioned/q95.plan.txt | 30 +- .../iceberg/orc/partitioned/q97.plan.txt | 26 +- .../iceberg/orc/partitioned/q98.plan.txt | 28 +- .../iceberg/orc/partitioned/q99.plan.txt | 19 +- .../iceberg/orc/unpartitioned/q01.plan.txt | 51 ++- .../iceberg/orc/unpartitioned/q02.plan.txt | 67 ++-- .../iceberg/orc/unpartitioned/q03.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q04.plan.txt | 180 +++++----- .../iceberg/orc/unpartitioned/q05.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q06.plan.txt | 43 +-- .../iceberg/orc/unpartitioned/q07.plan.txt | 19 +- .../iceberg/orc/unpartitioned/q08.plan.txt | 43 +-- .../iceberg/orc/unpartitioned/q10.plan.txt | 84 +++-- .../iceberg/orc/unpartitioned/q11.plan.txt | 120 +++---- .../iceberg/orc/unpartitioned/q12.plan.txt | 31 +- .../iceberg/orc/unpartitioned/q13.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q14.plan.txt | 324 +++++++++--------- .../iceberg/orc/unpartitioned/q15.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q16.plan.txt | 43 ++- .../iceberg/orc/unpartitioned/q17.plan.txt | 46 ++- .../iceberg/orc/unpartitioned/q18.plan.txt | 52 ++- .../iceberg/orc/unpartitioned/q19.plan.txt | 19 +- .../iceberg/orc/unpartitioned/q20.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q21.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q23.plan.txt | 140 ++++---- .../iceberg/orc/unpartitioned/q24.plan.txt | 75 ++-- .../iceberg/orc/unpartitioned/q25.plan.txt | 44 ++- .../iceberg/orc/unpartitioned/q26.plan.txt | 19 +- .../iceberg/orc/unpartitioned/q27.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q28.plan.txt | 18 +- .../iceberg/orc/unpartitioned/q29.plan.txt | 44 ++- .../iceberg/orc/unpartitioned/q30.plan.txt | 75 ++-- .../iceberg/orc/unpartitioned/q31.plan.txt | 151 ++++---- .../iceberg/orc/unpartitioned/q32.plan.txt | 24 +- .../iceberg/orc/unpartitioned/q33.plan.txt | 102 +++--- .../iceberg/orc/unpartitioned/q34.plan.txt | 33 +- .../iceberg/orc/unpartitioned/q35.plan.txt | 46 ++- .../iceberg/orc/unpartitioned/q36.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q37.plan.txt | 23 +- .../iceberg/orc/unpartitioned/q38.plan.txt | 42 +-- .../iceberg/orc/unpartitioned/q39.plan.txt | 60 ++-- .../iceberg/orc/unpartitioned/q40.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q41.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q42.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q43.plan.txt | 30 +- .../iceberg/orc/unpartitioned/q44.plan.txt | 60 ++-- .../iceberg/orc/unpartitioned/q45.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q46.plan.txt | 45 ++- .../iceberg/orc/unpartitioned/q47.plan.txt | 90 +++-- .../iceberg/orc/unpartitioned/q48.plan.txt | 17 +- .../iceberg/orc/unpartitioned/q49.plan.txt | 111 +++--- .../iceberg/orc/unpartitioned/q50.plan.txt | 14 +- .../iceberg/orc/unpartitioned/q51.plan.txt | 51 ++- .../iceberg/orc/unpartitioned/q52.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q53.plan.txt | 30 +- .../iceberg/orc/unpartitioned/q54.plan.txt | 117 +++---- .../iceberg/orc/unpartitioned/q55.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q56.plan.txt | 102 +++--- .../iceberg/orc/unpartitioned/q57.plan.txt | 90 +++-- .../iceberg/orc/unpartitioned/q58.plan.txt | 123 +++---- .../iceberg/orc/unpartitioned/q59.plan.txt | 63 ++-- .../iceberg/orc/unpartitioned/q60.plan.txt | 99 +++--- .../iceberg/orc/unpartitioned/q61.plan.txt | 52 ++- .../iceberg/orc/unpartitioned/q62.plan.txt | 19 +- .../iceberg/orc/unpartitioned/q63.plan.txt | 30 +- .../iceberg/orc/unpartitioned/q64.plan.txt | 318 ++++++++--------- .../iceberg/orc/unpartitioned/q65.plan.txt | 38 +- .../iceberg/orc/unpartitioned/q66.plan.txt | 68 ++-- .../iceberg/orc/unpartitioned/q68.plan.txt | 45 ++- .../iceberg/orc/unpartitioned/q69.plan.txt | 63 ++-- .../iceberg/orc/unpartitioned/q70.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q72.plan.txt | 32 +- .../iceberg/orc/unpartitioned/q73.plan.txt | 33 +- .../iceberg/orc/unpartitioned/q74.plan.txt | 120 +++---- .../iceberg/orc/unpartitioned/q75.plan.txt | 198 +++++------ .../iceberg/orc/unpartitioned/q76.plan.txt | 38 +- .../iceberg/orc/unpartitioned/q77.plan.txt | 86 +++-- .../iceberg/orc/unpartitioned/q78.plan.txt | 88 +++-- .../iceberg/orc/unpartitioned/q79.plan.txt | 33 +- .../iceberg/orc/unpartitioned/q80.plan.txt | 18 +- .../iceberg/orc/unpartitioned/q81.plan.txt | 75 ++-- .../iceberg/orc/unpartitioned/q82.plan.txt | 23 +- .../iceberg/orc/unpartitioned/q83.plan.txt | 144 ++++---- .../iceberg/orc/unpartitioned/q85.plan.txt | 65 ++-- .../iceberg/orc/unpartitioned/q86.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q87.plan.txt | 42 +-- .../iceberg/orc/unpartitioned/q89.plan.txt | 30 +- .../iceberg/orc/unpartitioned/q91.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q92.plan.txt | 24 +- .../iceberg/orc/unpartitioned/q93.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q94.plan.txt | 43 ++- .../iceberg/orc/unpartitioned/q95.plan.txt | 30 +- .../iceberg/orc/unpartitioned/q97.plan.txt | 26 +- .../iceberg/orc/unpartitioned/q98.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q99.plan.txt | 19 +- .../iceberg/parquet/partitioned/q01.plan.txt | 51 ++- .../iceberg/parquet/partitioned/q02.plan.txt | 67 ++-- .../iceberg/parquet/partitioned/q03.plan.txt | 15 +- .../iceberg/parquet/partitioned/q04.plan.txt | 180 +++++----- .../iceberg/parquet/partitioned/q05.plan.txt | 6 +- .../iceberg/parquet/partitioned/q06.plan.txt | 43 +-- .../iceberg/parquet/partitioned/q07.plan.txt | 19 +- .../iceberg/parquet/partitioned/q08.plan.txt | 43 +-- .../iceberg/parquet/partitioned/q10.plan.txt | 84 +++-- .../iceberg/parquet/partitioned/q11.plan.txt | 120 +++---- .../iceberg/parquet/partitioned/q12.plan.txt | 28 +- .../iceberg/parquet/partitioned/q14.plan.txt | 324 +++++++++--------- .../iceberg/parquet/partitioned/q15.plan.txt | 17 +- .../iceberg/parquet/partitioned/q16.plan.txt | 43 ++- .../iceberg/parquet/partitioned/q17.plan.txt | 52 ++- .../iceberg/parquet/partitioned/q18.plan.txt | 9 +- .../iceberg/parquet/partitioned/q19.plan.txt | 19 +- .../iceberg/parquet/partitioned/q20.plan.txt | 28 +- .../iceberg/parquet/partitioned/q21.plan.txt | 17 +- .../iceberg/parquet/partitioned/q23.plan.txt | 140 ++++---- .../iceberg/parquet/partitioned/q24.plan.txt | 81 ++--- .../iceberg/parquet/partitioned/q25.plan.txt | 46 ++- .../iceberg/parquet/partitioned/q26.plan.txt | 19 +- .../iceberg/parquet/partitioned/q27.plan.txt | 18 +- .../iceberg/parquet/partitioned/q28.plan.txt | 18 +- .../iceberg/parquet/partitioned/q29.plan.txt | 46 ++- .../iceberg/parquet/partitioned/q30.plan.txt | 75 ++-- .../iceberg/parquet/partitioned/q31.plan.txt | 151 ++++---- .../iceberg/parquet/partitioned/q32.plan.txt | 24 +- .../iceberg/parquet/partitioned/q33.plan.txt | 102 +++--- .../iceberg/parquet/partitioned/q34.plan.txt | 33 +- .../iceberg/parquet/partitioned/q35.plan.txt | 46 ++- .../iceberg/parquet/partitioned/q36.plan.txt | 17 +- .../iceberg/parquet/partitioned/q37.plan.txt | 23 +- .../iceberg/parquet/partitioned/q38.plan.txt | 42 +-- .../iceberg/parquet/partitioned/q39.plan.txt | 60 ++-- .../iceberg/parquet/partitioned/q40.plan.txt | 6 +- .../iceberg/parquet/partitioned/q41.plan.txt | 15 +- .../iceberg/parquet/partitioned/q42.plan.txt | 15 +- .../iceberg/parquet/partitioned/q43.plan.txt | 30 +- .../iceberg/parquet/partitioned/q44.plan.txt | 60 ++-- .../iceberg/parquet/partitioned/q45.plan.txt | 19 +- .../iceberg/parquet/partitioned/q46.plan.txt | 45 ++- .../iceberg/parquet/partitioned/q47.plan.txt | 90 +++-- .../iceberg/parquet/partitioned/q49.plan.txt | 111 +++--- .../iceberg/parquet/partitioned/q50.plan.txt | 14 +- .../iceberg/parquet/partitioned/q51.plan.txt | 51 ++- .../iceberg/parquet/partitioned/q52.plan.txt | 15 +- .../iceberg/parquet/partitioned/q53.plan.txt | 30 +- .../iceberg/parquet/partitioned/q54.plan.txt | 117 +++---- .../iceberg/parquet/partitioned/q55.plan.txt | 15 +- .../iceberg/parquet/partitioned/q56.plan.txt | 100 +++--- .../iceberg/parquet/partitioned/q57.plan.txt | 90 +++-- .../iceberg/parquet/partitioned/q58.plan.txt | 119 +++---- .../iceberg/parquet/partitioned/q59.plan.txt | 63 ++-- .../iceberg/parquet/partitioned/q60.plan.txt | 102 +++--- .../iceberg/parquet/partitioned/q61.plan.txt | 52 ++- .../iceberg/parquet/partitioned/q62.plan.txt | 19 +- .../iceberg/parquet/partitioned/q63.plan.txt | 30 +- .../iceberg/parquet/partitioned/q64.plan.txt | 310 ++++++++--------- .../iceberg/parquet/partitioned/q65.plan.txt | 38 +- .../iceberg/parquet/partitioned/q66.plan.txt | 68 ++-- .../iceberg/parquet/partitioned/q68.plan.txt | 45 ++- .../iceberg/parquet/partitioned/q69.plan.txt | 93 +++-- .../iceberg/parquet/partitioned/q70.plan.txt | 28 +- .../iceberg/parquet/partitioned/q72.plan.txt | 32 +- .../iceberg/parquet/partitioned/q73.plan.txt | 33 +- .../iceberg/parquet/partitioned/q74.plan.txt | 120 +++---- .../iceberg/parquet/partitioned/q75.plan.txt | 198 +++++------ .../iceberg/parquet/partitioned/q76.plan.txt | 39 +-- .../iceberg/parquet/partitioned/q77.plan.txt | 86 +++-- .../iceberg/parquet/partitioned/q78.plan.txt | 88 +++-- .../iceberg/parquet/partitioned/q79.plan.txt | 33 +- .../iceberg/parquet/partitioned/q80.plan.txt | 18 +- .../iceberg/parquet/partitioned/q81.plan.txt | 47 ++- .../iceberg/parquet/partitioned/q82.plan.txt | 23 +- .../iceberg/parquet/partitioned/q83.plan.txt | 144 ++++---- .../iceberg/parquet/partitioned/q85.plan.txt | 62 ++-- .../iceberg/parquet/partitioned/q86.plan.txt | 15 +- .../iceberg/parquet/partitioned/q87.plan.txt | 42 +-- .../iceberg/parquet/partitioned/q89.plan.txt | 30 +- .../iceberg/parquet/partitioned/q91.plan.txt | 45 ++- .../iceberg/parquet/partitioned/q92.plan.txt | 24 +- .../iceberg/parquet/partitioned/q93.plan.txt | 6 +- .../iceberg/parquet/partitioned/q94.plan.txt | 43 ++- .../iceberg/parquet/partitioned/q95.plan.txt | 30 +- .../iceberg/parquet/partitioned/q97.plan.txt | 26 +- .../iceberg/parquet/partitioned/q98.plan.txt | 28 +- .../iceberg/parquet/partitioned/q99.plan.txt | 19 +- .../parquet/unpartitioned/q01.plan.txt | 51 ++- .../parquet/unpartitioned/q02.plan.txt | 67 ++-- .../parquet/unpartitioned/q03.plan.txt | 15 +- .../parquet/unpartitioned/q04.plan.txt | 180 +++++----- .../parquet/unpartitioned/q05.plan.txt | 6 +- .../parquet/unpartitioned/q06.plan.txt | 43 +-- .../parquet/unpartitioned/q07.plan.txt | 19 +- .../parquet/unpartitioned/q08.plan.txt | 43 +-- .../parquet/unpartitioned/q10.plan.txt | 84 +++-- .../parquet/unpartitioned/q11.plan.txt | 120 +++---- .../parquet/unpartitioned/q12.plan.txt | 28 +- .../parquet/unpartitioned/q14.plan.txt | 324 +++++++++--------- .../parquet/unpartitioned/q15.plan.txt | 17 +- .../parquet/unpartitioned/q16.plan.txt | 43 ++- .../parquet/unpartitioned/q17.plan.txt | 52 ++- .../parquet/unpartitioned/q18.plan.txt | 9 +- .../parquet/unpartitioned/q19.plan.txt | 19 +- .../parquet/unpartitioned/q20.plan.txt | 28 +- .../parquet/unpartitioned/q21.plan.txt | 17 +- .../parquet/unpartitioned/q23.plan.txt | 140 ++++---- .../parquet/unpartitioned/q24.plan.txt | 81 ++--- .../parquet/unpartitioned/q25.plan.txt | 46 ++- .../parquet/unpartitioned/q26.plan.txt | 19 +- .../parquet/unpartitioned/q27.plan.txt | 18 +- .../parquet/unpartitioned/q28.plan.txt | 18 +- .../parquet/unpartitioned/q29.plan.txt | 46 ++- .../parquet/unpartitioned/q30.plan.txt | 75 ++-- .../parquet/unpartitioned/q31.plan.txt | 151 ++++---- .../parquet/unpartitioned/q32.plan.txt | 24 +- .../parquet/unpartitioned/q33.plan.txt | 102 +++--- .../parquet/unpartitioned/q34.plan.txt | 33 +- .../parquet/unpartitioned/q35.plan.txt | 46 ++- .../parquet/unpartitioned/q36.plan.txt | 17 +- .../parquet/unpartitioned/q37.plan.txt | 23 +- .../parquet/unpartitioned/q38.plan.txt | 42 +-- .../parquet/unpartitioned/q39.plan.txt | 60 ++-- .../parquet/unpartitioned/q40.plan.txt | 6 +- .../parquet/unpartitioned/q41.plan.txt | 15 +- .../parquet/unpartitioned/q42.plan.txt | 15 +- .../parquet/unpartitioned/q43.plan.txt | 30 +- .../parquet/unpartitioned/q44.plan.txt | 60 ++-- .../parquet/unpartitioned/q45.plan.txt | 19 +- .../parquet/unpartitioned/q46.plan.txt | 45 ++- .../parquet/unpartitioned/q47.plan.txt | 90 +++-- .../parquet/unpartitioned/q49.plan.txt | 111 +++--- .../parquet/unpartitioned/q50.plan.txt | 14 +- .../parquet/unpartitioned/q51.plan.txt | 51 ++- .../parquet/unpartitioned/q52.plan.txt | 15 +- .../parquet/unpartitioned/q53.plan.txt | 30 +- .../parquet/unpartitioned/q54.plan.txt | 117 +++---- .../parquet/unpartitioned/q55.plan.txt | 15 +- .../parquet/unpartitioned/q56.plan.txt | 100 +++--- .../parquet/unpartitioned/q57.plan.txt | 90 +++-- .../parquet/unpartitioned/q58.plan.txt | 119 +++---- .../parquet/unpartitioned/q59.plan.txt | 63 ++-- .../parquet/unpartitioned/q60.plan.txt | 102 +++--- .../parquet/unpartitioned/q61.plan.txt | 52 ++- .../parquet/unpartitioned/q62.plan.txt | 19 +- .../parquet/unpartitioned/q63.plan.txt | 30 +- .../parquet/unpartitioned/q64.plan.txt | 310 ++++++++--------- .../parquet/unpartitioned/q65.plan.txt | 38 +- .../parquet/unpartitioned/q66.plan.txt | 68 ++-- .../parquet/unpartitioned/q68.plan.txt | 45 ++- .../parquet/unpartitioned/q69.plan.txt | 93 +++-- .../parquet/unpartitioned/q70.plan.txt | 28 +- .../parquet/unpartitioned/q72.plan.txt | 32 +- .../parquet/unpartitioned/q73.plan.txt | 33 +- .../parquet/unpartitioned/q74.plan.txt | 120 +++---- .../parquet/unpartitioned/q75.plan.txt | 198 +++++------ .../parquet/unpartitioned/q76.plan.txt | 39 +-- .../parquet/unpartitioned/q77.plan.txt | 86 +++-- .../parquet/unpartitioned/q78.plan.txt | 88 +++-- .../parquet/unpartitioned/q79.plan.txt | 33 +- .../parquet/unpartitioned/q80.plan.txt | 18 +- .../parquet/unpartitioned/q81.plan.txt | 47 ++- .../parquet/unpartitioned/q82.plan.txt | 23 +- .../parquet/unpartitioned/q83.plan.txt | 144 ++++---- .../parquet/unpartitioned/q85.plan.txt | 62 ++-- .../parquet/unpartitioned/q86.plan.txt | 15 +- .../parquet/unpartitioned/q87.plan.txt | 42 +-- .../parquet/unpartitioned/q89.plan.txt | 30 +- .../parquet/unpartitioned/q91.plan.txt | 45 ++- .../parquet/unpartitioned/q92.plan.txt | 24 +- .../parquet/unpartitioned/q93.plan.txt | 6 +- .../parquet/unpartitioned/q94.plan.txt | 43 ++- .../parquet/unpartitioned/q95.plan.txt | 30 +- .../parquet/unpartitioned/q97.plan.txt | 26 +- .../parquet/unpartitioned/q98.plan.txt | 28 +- .../parquet/unpartitioned/q99.plan.txt | 19 +- .../presto/tpch/hive/partitioned/q01.plan.txt | 5 +- .../presto/tpch/hive/partitioned/q02.plan.txt | 57 ++- .../presto/tpch/hive/partitioned/q03.plan.txt | 9 +- .../presto/tpch/hive/partitioned/q04.plan.txt | 22 +- .../presto/tpch/hive/partitioned/q05.plan.txt | 33 +- .../presto/tpch/hive/partitioned/q07.plan.txt | 29 +- .../presto/tpch/hive/partitioned/q08.plan.txt | 42 +-- .../presto/tpch/hive/partitioned/q09.plan.txt | 15 +- .../presto/tpch/hive/partitioned/q10.plan.txt | 9 +- .../presto/tpch/hive/partitioned/q11.plan.txt | 21 +- .../presto/tpch/hive/partitioned/q12.plan.txt | 6 +- .../presto/tpch/hive/partitioned/q13.plan.txt | 23 +- .../presto/tpch/hive/partitioned/q14.plan.txt | 6 +- .../presto/tpch/hive/partitioned/q15.plan.txt | 10 +- .../presto/tpch/hive/partitioned/q16.plan.txt | 34 +- .../presto/tpch/hive/partitioned/q17.plan.txt | 5 +- .../presto/tpch/hive/partitioned/q18.plan.txt | 28 +- .../presto/tpch/hive/partitioned/q19.plan.txt | 6 +- .../presto/tpch/hive/partitioned/q20.plan.txt | 55 ++- .../presto/tpch/hive/partitioned/q21.plan.txt | 55 ++- .../presto/tpch/hive/partitioned/q22.plan.txt | 24 +- .../tpch/hive/unpartitioned/q01.plan.txt | 5 +- .../tpch/hive/unpartitioned/q02.plan.txt | 57 ++- .../tpch/hive/unpartitioned/q03.plan.txt | 9 +- .../tpch/hive/unpartitioned/q04.plan.txt | 21 +- .../tpch/hive/unpartitioned/q05.plan.txt | 33 +- .../tpch/hive/unpartitioned/q07.plan.txt | 33 +- .../tpch/hive/unpartitioned/q08.plan.txt | 42 +-- .../tpch/hive/unpartitioned/q09.plan.txt | 15 +- .../tpch/hive/unpartitioned/q10.plan.txt | 9 +- .../tpch/hive/unpartitioned/q11.plan.txt | 21 +- .../tpch/hive/unpartitioned/q12.plan.txt | 6 +- .../tpch/hive/unpartitioned/q13.plan.txt | 23 +- .../tpch/hive/unpartitioned/q14.plan.txt | 6 +- .../tpch/hive/unpartitioned/q15.plan.txt | 13 +- .../tpch/hive/unpartitioned/q16.plan.txt | 34 +- .../tpch/hive/unpartitioned/q17.plan.txt | 16 +- .../tpch/hive/unpartitioned/q18.plan.txt | 14 +- .../tpch/hive/unpartitioned/q19.plan.txt | 6 +- .../tpch/hive/unpartitioned/q20.plan.txt | 55 ++- .../tpch/hive/unpartitioned/q21.plan.txt | 61 ++-- .../tpch/hive/unpartitioned/q22.plan.txt | 24 +- .../tpch/iceberg/orc/partitioned/q01.plan.txt | 5 +- .../tpch/iceberg/orc/partitioned/q02.plan.txt | 54 ++- .../tpch/iceberg/orc/partitioned/q03.plan.txt | 9 +- .../tpch/iceberg/orc/partitioned/q04.plan.txt | 21 +- .../tpch/iceberg/orc/partitioned/q05.plan.txt | 28 +- .../tpch/iceberg/orc/partitioned/q07.plan.txt | 33 +- .../tpch/iceberg/orc/partitioned/q08.plan.txt | 34 +- .../tpch/iceberg/orc/partitioned/q09.plan.txt | 15 +- .../tpch/iceberg/orc/partitioned/q10.plan.txt | 9 +- .../tpch/iceberg/orc/partitioned/q11.plan.txt | 21 +- .../tpch/iceberg/orc/partitioned/q12.plan.txt | 6 +- .../tpch/iceberg/orc/partitioned/q13.plan.txt | 23 +- .../tpch/iceberg/orc/partitioned/q14.plan.txt | 6 +- .../tpch/iceberg/orc/partitioned/q15.plan.txt | 13 +- .../tpch/iceberg/orc/partitioned/q16.plan.txt | 34 +- .../tpch/iceberg/orc/partitioned/q17.plan.txt | 16 +- .../tpch/iceberg/orc/partitioned/q18.plan.txt | 14 +- .../tpch/iceberg/orc/partitioned/q19.plan.txt | 6 +- .../tpch/iceberg/orc/partitioned/q20.plan.txt | 55 ++- .../tpch/iceberg/orc/partitioned/q21.plan.txt | 57 ++- .../tpch/iceberg/orc/partitioned/q22.plan.txt | 24 +- .../iceberg/orc/unpartitioned/q01.plan.txt | 5 +- .../iceberg/orc/unpartitioned/q02.plan.txt | 54 ++- .../iceberg/orc/unpartitioned/q03.plan.txt | 9 +- .../iceberg/orc/unpartitioned/q04.plan.txt | 21 +- .../iceberg/orc/unpartitioned/q05.plan.txt | 28 +- .../iceberg/orc/unpartitioned/q07.plan.txt | 33 +- .../iceberg/orc/unpartitioned/q08.plan.txt | 34 +- .../iceberg/orc/unpartitioned/q09.plan.txt | 15 +- .../iceberg/orc/unpartitioned/q10.plan.txt | 9 +- .../iceberg/orc/unpartitioned/q11.plan.txt | 21 +- .../iceberg/orc/unpartitioned/q12.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q13.plan.txt | 23 +- .../iceberg/orc/unpartitioned/q14.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q15.plan.txt | 13 +- .../iceberg/orc/unpartitioned/q16.plan.txt | 34 +- .../iceberg/orc/unpartitioned/q17.plan.txt | 16 +- .../iceberg/orc/unpartitioned/q18.plan.txt | 14 +- .../iceberg/orc/unpartitioned/q19.plan.txt | 6 +- .../iceberg/orc/unpartitioned/q20.plan.txt | 55 ++- .../iceberg/orc/unpartitioned/q21.plan.txt | 57 ++- .../iceberg/orc/unpartitioned/q22.plan.txt | 24 +- .../iceberg/parquet/partitioned/q01.plan.txt | 5 +- .../iceberg/parquet/partitioned/q02.plan.txt | 57 ++- .../iceberg/parquet/partitioned/q03.plan.txt | 9 +- .../iceberg/parquet/partitioned/q04.plan.txt | 21 +- .../iceberg/parquet/partitioned/q05.plan.txt | 28 +- .../iceberg/parquet/partitioned/q07.plan.txt | 33 +- .../iceberg/parquet/partitioned/q08.plan.txt | 34 +- .../iceberg/parquet/partitioned/q09.plan.txt | 15 +- .../iceberg/parquet/partitioned/q10.plan.txt | 9 +- .../iceberg/parquet/partitioned/q11.plan.txt | 21 +- .../iceberg/parquet/partitioned/q12.plan.txt | 6 +- .../iceberg/parquet/partitioned/q13.plan.txt | 23 +- .../iceberg/parquet/partitioned/q14.plan.txt | 6 +- .../iceberg/parquet/partitioned/q15.plan.txt | 13 +- .../iceberg/parquet/partitioned/q16.plan.txt | 34 +- .../iceberg/parquet/partitioned/q17.plan.txt | 16 +- .../iceberg/parquet/partitioned/q18.plan.txt | 14 +- .../iceberg/parquet/partitioned/q19.plan.txt | 6 +- .../iceberg/parquet/partitioned/q20.plan.txt | 55 ++- .../iceberg/parquet/partitioned/q21.plan.txt | 61 ++-- .../iceberg/parquet/partitioned/q22.plan.txt | 24 +- .../parquet/unpartitioned/q01.plan.txt | 5 +- .../parquet/unpartitioned/q02.plan.txt | 57 ++- .../parquet/unpartitioned/q03.plan.txt | 9 +- .../parquet/unpartitioned/q04.plan.txt | 21 +- .../parquet/unpartitioned/q05.plan.txt | 28 +- .../parquet/unpartitioned/q07.plan.txt | 33 +- .../parquet/unpartitioned/q08.plan.txt | 34 +- .../parquet/unpartitioned/q09.plan.txt | 15 +- .../parquet/unpartitioned/q10.plan.txt | 9 +- .../parquet/unpartitioned/q11.plan.txt | 21 +- .../parquet/unpartitioned/q12.plan.txt | 6 +- .../parquet/unpartitioned/q13.plan.txt | 23 +- .../parquet/unpartitioned/q14.plan.txt | 6 +- .../parquet/unpartitioned/q15.plan.txt | 13 +- .../parquet/unpartitioned/q16.plan.txt | 34 +- .../parquet/unpartitioned/q17.plan.txt | 16 +- .../parquet/unpartitioned/q18.plan.txt | 14 +- .../parquet/unpartitioned/q19.plan.txt | 6 +- .../parquet/unpartitioned/q20.plan.txt | 55 ++- .../parquet/unpartitioned/q21.plan.txt | 61 ++-- .../parquet/unpartitioned/q22.plan.txt | 24 +- 687 files changed, 15625 insertions(+), 18838 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index 1e736fd60a52..6959ed5ff27e 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -177,7 +177,6 @@ public final class SystemSessionProperties public static final String USE_EXACT_PARTITIONING = "use_exact_partitioning"; public static final String FORCE_SPILLING_JOIN = "force_spilling_join"; public static final String FAULT_TOLERANT_EXECUTION_EVENT_DRIVEN_SCHEDULER_ENABLED = "fault_tolerant_execution_event_driven_scheduler_enabled"; - public static final String FORCE_FIXED_DISTRIBUTION_FOR_PARTITIONED_OUTPUT_OPERATOR_ENABLED = "force_fixed_distribution_for_partitioned_output_operator_enabled"; public static final String FAULT_TOLERANT_EXECUTION_FORCE_PREFERRED_WRITE_PARTITIONING_ENABLED = "fault_tolerant_execution_force_preferred_write_partitioning_enabled"; private final List> sessionProperties; @@ -879,11 +878,6 @@ public SystemSessionProperties( "Enable event driven scheduler for fault tolerant execution", queryManagerConfig.isFaultTolerantExecutionEventDrivenSchedulerEnabled(), true), - booleanProperty( - FORCE_FIXED_DISTRIBUTION_FOR_PARTITIONED_OUTPUT_OPERATOR_ENABLED, - "Force partitioned output operator to be run with fixed distribution", - optimizerConfig.isForceFixedDistributionForPartitionedOutputOperatorEnabled(), - true), booleanProperty( FAULT_TOLERANT_EXECUTION_FORCE_PREFERRED_WRITE_PARTITIONING_ENABLED, "Force preferred write partitioning for fault tolerant execution", @@ -1572,11 +1566,6 @@ public static boolean isFaultTolerantExecutionEventDriverSchedulerEnabled(Sessio return session.getSystemProperty(FAULT_TOLERANT_EXECUTION_EVENT_DRIVEN_SCHEDULER_ENABLED, Boolean.class); } - public static boolean isForceFixedDistributionForPartitionedOutputOperatorEnabled(Session session) - { - return session.getSystemProperty(FORCE_FIXED_DISTRIBUTION_FOR_PARTITIONED_OUTPUT_OPERATOR_ENABLED, Boolean.class); - } - public static boolean isFaultTolerantExecutionForcePreferredWritePartitioningEnabled(Session session) { if (!isFaultTolerantExecutionEventDriverSchedulerEnabled(session)) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java b/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java index 164ee812236f..faf821a80da4 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java @@ -89,8 +89,6 @@ public class OptimizerConfig private double adaptivePartialAggregationUniqueRowsRatioThreshold = 0.8; private long joinPartitionedBuildMinRowCount = 1_000_000L; - private boolean forceFixedDistributionForPartitionedOutputOperatorEnabled = true; - public enum JoinReorderingStrategy { NONE, @@ -758,16 +756,4 @@ public OptimizerConfig setUseExactPartitioning(boolean useExactPartitioning) this.useExactPartitioning = useExactPartitioning; return this; } - - public boolean isForceFixedDistributionForPartitionedOutputOperatorEnabled() - { - return forceFixedDistributionForPartitionedOutputOperatorEnabled; - } - - @Config("experimental.force-fixed-distribution-for-partitioned-output-operator-enabled") - public OptimizerConfig setForceFixedDistributionForPartitionedOutputOperatorEnabled(boolean forceFixedDistributionForPartitionedOutputOperatorEnabled) - { - this.forceFixedDistributionForPartitionedOutputOperatorEnabled = forceFixedDistributionForPartitionedOutputOperatorEnabled; - return this; - } } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AddExchangesBelowPartialAggregationOverGroupIdRuleSet.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AddExchangesBelowPartialAggregationOverGroupIdRuleSet.java index 3926d4805561..a666f6daeec5 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AddExchangesBelowPartialAggregationOverGroupIdRuleSet.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/AddExchangesBelowPartialAggregationOverGroupIdRuleSet.java @@ -104,33 +104,25 @@ public class AddExchangesBelowPartialAggregationOverGroupIdRuleSet typeOf(ExchangeNode.class) .with(scope().equalTo(REMOTE)) .with(source().matching( - typeOf(ProjectNode.class) + // PushPartialAggregationThroughExchange adds a projection. However, it can be removed if RemoveRedundantIdentityProjections is run in the mean-time. + typeOf(ProjectNode.class).capturedAs(PROJECTION) .with(source().matching( - typeOf(ExchangeNode.class) - .with(scope().equalTo(LOCAL)) + typeOf(AggregationNode.class).capturedAs(AGGREGATION) + .with(step().equalTo(AggregationNode.Step.PARTIAL)) + .with(nonEmpty(groupingColumns())) .with(source().matching( - // PushPartialAggregationThroughExchange adds a projection. However, it can be removed if RemoveRedundantIdentityProjections is run in the mean-time. - typeOf(ProjectNode.class).capturedAs(PROJECTION) - .with(source().matching( - typeOf(AggregationNode.class).capturedAs(AGGREGATION) - .with(step().equalTo(AggregationNode.Step.PARTIAL)) - .with(nonEmpty(groupingColumns())) - .with(source().matching( - typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))))))))); + typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))))); private static final Pattern WITHOUT_PROJECTION = // If there was no exchange here, adding new exchanges could break property derivations logic of AddExchanges, AddLocalExchanges typeOf(ExchangeNode.class) .with(scope().equalTo(REMOTE)) .with(source().matching( - typeOf(ExchangeNode.class) - .with(scope().equalTo(LOCAL)) + typeOf(AggregationNode.class).capturedAs(AGGREGATION) + .with(step().equalTo(AggregationNode.Step.PARTIAL)) + .with(nonEmpty(groupingColumns())) .with(source().matching( - typeOf(AggregationNode.class).capturedAs(AGGREGATION) - .with(step().equalTo(AggregationNode.Step.PARTIAL)) - .with(nonEmpty(groupingColumns())) - .with(source().matching( - typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))))); + typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))); private static final double GROUPING_SETS_SYMBOL_REQUIRED_FREQUENCY = 0.5; private static final double ANTI_SKEWNESS_MARGIN = 3; diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java index e7291d355e46..6c3ae07e99c9 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/AddLocalExchanges.java @@ -80,7 +80,6 @@ import static io.trino.SystemSessionProperties.getTaskPartitionedWriterCount; import static io.trino.SystemSessionProperties.getTaskWriterCount; import static io.trino.SystemSessionProperties.isDistributedSortEnabled; -import static io.trino.SystemSessionProperties.isForceFixedDistributionForPartitionedOutputOperatorEnabled; import static io.trino.SystemSessionProperties.isSpillEnabled; import static io.trino.SystemSessionProperties.isTaskScaleWritersEnabled; import static io.trino.sql.ExpressionUtils.isEffectivelyLiteral; @@ -744,9 +743,6 @@ public PlanWithProperties visitExchange(ExchangeNode node, StreamPreferredProper any().withOrderSensitivity(), any().withOrderSensitivity()); } - if (isForceFixedDistributionForPartitionedOutputOperatorEnabled(session) && node.isHashPartitionedExchange()) { - return planAndEnforceChildren(node, fixedParallelism(), defaultParallelism(session)); - } return planAndEnforceChildren(node, any(), defaultParallelism(session)); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/ExchangeNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/ExchangeNode.java index 71c0d88b409b..f5b8fe0fe3b2 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/ExchangeNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/ExchangeNode.java @@ -34,7 +34,6 @@ import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_PASSTHROUGH_DISTRIBUTION; -import static io.trino.sql.planner.SystemPartitioningHandle.SCALED_WRITER_HASH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.plan.ExchangeNode.Scope.LOCAL; import static io.trino.sql.planner.plan.ExchangeNode.Scope.REMOTE; @@ -251,11 +250,4 @@ public PlanNode replaceChildren(List newChildren) { return new ExchangeNode(getId(), type, scope, partitioningScheme, newChildren, inputs, orderingScheme); } - - public boolean isHashPartitionedExchange() - { - PartitioningHandle partitioningHandle = partitioningScheme.getPartitioning().getHandle(); - // catalog specific exchanges are hash partitioned exchanges with a catalog specific partition function - return partitioningHandle.equals(FIXED_HASH_DISTRIBUTION) || partitioningHandle.equals(SCALED_WRITER_HASH_DISTRIBUTION) || partitioningHandle.getCatalogHandle().isPresent(); - } } diff --git a/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java b/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java index 6e2b5e791e95..12306998472d 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java @@ -89,8 +89,7 @@ public void testDefaults() .setAdaptivePartialAggregationMinRows(100_000) .setAdaptivePartialAggregationUniqueRowsRatioThreshold(0.8) .setJoinPartitionedBuildMinRowCount(1_000_000) - .setUseExactPartitioning(false) - .setForceFixedDistributionForPartitionedOutputOperatorEnabled(true)); + .setUseExactPartitioning(false)); } @Test @@ -148,7 +147,6 @@ public void testExplicitPropertyMappings() .put("adaptive-partial-aggregation.unique-rows-ratio-threshold", "0.99") .put("optimizer.join-partitioned-build-min-row-count", "1") .put("optimizer.use-exact-partitioning", "true") - .put("experimental.force-fixed-distribution-for-partitioned-output-operator-enabled", "false") .buildOrThrow(); OptimizerConfig expected = new OptimizerConfig() @@ -202,8 +200,7 @@ public void testExplicitPropertyMappings() .setAdaptivePartialAggregationMinRows(1) .setAdaptivePartialAggregationUniqueRowsRatioThreshold(0.99) .setJoinPartitionedBuildMinRowCount(1) - .setUseExactPartitioning(true) - .setForceFixedDistributionForPartitionedOutputOperatorEnabled(false); + .setUseExactPartitioning(true); assertFullMapping(properties, expected); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java index 40335f3730fd..f87227b16042 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestAddDynamicFilterSource.java @@ -31,6 +31,7 @@ import io.trino.sql.planner.plan.FilterNode; import io.trino.sql.planner.plan.PlanNode; import io.trino.sql.planner.plan.SemiJoinNode; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.Optional; @@ -72,12 +73,12 @@ public TestAddDynamicFilterSource() JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.NONE.name())); } - @Test - public void testPartitionedInnerJoin() + @Test(dataProvider = "joinDistributionTypes") + public void testInnerJoin(JoinDistributionType joinDistributionType) { assertDistributedPlan( "SELECT l.suppkey FROM lineitem l, supplier s WHERE l.suppkey = s.suppkey", - withJoinDistributionType(PARTITIONED), + withJoinDistributionType(joinDistributionType), anyTree( join(INNER, builder -> builder .equiCriteria("LINEITEM_SK", "SUPPLIER_SK") @@ -93,99 +94,41 @@ public void testPartitionedInnerJoin() LOCAL, exchange( REMOTE, - REPARTITION, - node( - DynamicFilterSourceNode.class, - exchange( - LOCAL, - project( - tableScan("supplier", ImmutableMap.of("SUPPLIER_SK", "suppkey"))))))))))); - } - - @Test - public void testBroadcastInnerJoin() - { - assertDistributedPlan( - "SELECT l.suppkey FROM lineitem l, supplier s WHERE l.suppkey = s.suppkey", - withJoinDistributionType(BROADCAST), - anyTree( - join(INNER, builder -> builder - .equiCriteria("LINEITEM_SK", "SUPPLIER_SK") - .dynamicFilter("LINEITEM_SK", "SUPPLIER_SK") - .left( - anyTree( - node( - FilterNode.class, - tableScan("lineitem", ImmutableMap.of("LINEITEM_SK", "suppkey"))) - .with(numberOfDynamicFilters(1)))) - .right( - exchange( - LOCAL, - exchange( - REMOTE, - REPLICATE, + joinDistributionType == PARTITIONED ? REPARTITION : REPLICATE, node( DynamicFilterSourceNode.class, project( tableScan("supplier", ImmutableMap.of("SUPPLIER_SK", "suppkey")))))))))); } - @Test - public void testPartitionedSemiJoin() + @Test(dataProvider = "joinDistributionTypes") + public void testSemiJoin(JoinDistributionType joinDistributionType) { + SemiJoinNode.DistributionType semiJoinDistributionType = joinDistributionType == PARTITIONED + ? SemiJoinNode.DistributionType.PARTITIONED + : SemiJoinNode.DistributionType.REPLICATED; assertDistributedPlan( "SELECT * FROM orders WHERE orderkey IN (SELECT orderkey FROM lineitem WHERE linenumber % 4 = 0)", - noSemiJoinRewrite(PARTITIONED), + noSemiJoinRewrite(joinDistributionType), anyTree( filter("S", project( - semiJoin("X", "Y", "S", Optional.of(SemiJoinNode.DistributionType.PARTITIONED), Optional.of(true), + semiJoin("X", "Y", "S", Optional.of(semiJoinDistributionType), Optional.of(true), anyTree( node( FilterNode.class, tableScan("orders", ImmutableMap.of("X", "orderkey"))) - .with(numberOfDynamicFilters(1))), - exchange( - LOCAL, + .with(numberOfDynamicFilters(1))), exchange( - REMOTE, - REPARTITION, - node( - DynamicFilterSourceNode.class, - exchange( - LOCAL, - REPARTITION, + LOCAL, + exchange( + REMOTE, + joinDistributionType == PARTITIONED ? REPARTITION : REPLICATE, + node( + DynamicFilterSourceNode.class, project( filter( "Z % 4 = 0", - tableScan("lineitem", ImmutableMap.of("Y", "orderkey", "Z", "linenumber"))))))))))))); - } - - @Test - public void testBroadcastSemiJoin() - { - assertDistributedPlan( - "SELECT * FROM orders WHERE orderkey IN (SELECT orderkey FROM lineitem WHERE linenumber % 4 = 0)", - noSemiJoinRewrite(BROADCAST), - anyTree( - filter("S", - project( - semiJoin("X", "Y", "S", Optional.of(SemiJoinNode.DistributionType.REPLICATED), Optional.of(true), - anyTree( - node( - FilterNode.class, - tableScan("orders", ImmutableMap.of("X", "orderkey"))) - .with(numberOfDynamicFilters(1))), - exchange( - LOCAL, - exchange( - REMOTE, - REPLICATE, - node( - DynamicFilterSourceNode.class, - project( - filter( - "Z % 4 = 0", tableScan("lineitem", ImmutableMap.of("Y", "orderkey", "Z", "linenumber")))))))))))); } @@ -240,8 +183,8 @@ public void testInnerJoinWithUnionAllOnBuild() ImmutableSet.of(), Optional.empty(), ImmutableList.of("SUPPLIER_SK"), - exchange(exchange(LOCAL, project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_1", "suppkey"))))), - exchange(exchange(LOCAL, project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_2", "suppkey")))))))))); + exchange(project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_1", "suppkey")))), + exchange(project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_2", "suppkey"))))))))); } @Test @@ -300,11 +243,8 @@ public void testJoinWithPrePartitionedBuild() exchange( REMOTE, REPARTITION, - exchange( - LOCAL, - REPARTITION, - project( - tableScan("lineitem", ImmutableMap.of("LINEITEM_SK", "suppkey")))))) + project( + tableScan("lineitem", ImmutableMap.of("LINEITEM_SK", "suppkey"))))) .right( anyTree( tableScan("supplier", ImmutableMap.of("SUPPLIER_SK", "suppkey"))))))); @@ -329,6 +269,12 @@ public MatchResult detailMatches(PlanNode node, StatsProvider stats, Session ses }; } + @DataProvider + public Object[][] joinDistributionTypes() + { + return new Object[][] {{BROADCAST}, {PARTITIONED}}; + } + private Session noSemiJoinRewrite(JoinDistributionType distributionType) { return Session.builder(getQueryRunner().getDefaultSession()) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestInsert.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestInsert.java index 9102f9ada5e1..75ba685a9c8b 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestInsert.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestInsert.java @@ -153,8 +153,7 @@ private void testInsertWithRequiredPartitioning(Session session) node(TableWriterNode.class, exchange(LOCAL, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), - exchange(LOCAL, REPARTITION, - values("column1", "column2"))) + values("column1", "column2")) .with(exchangeWithoutSystemPartitioning())) .with(exchangeWithoutSystemPartitioning())))); } @@ -216,8 +215,7 @@ private void testCreateTableAsSelectWithRequiredPartitioning(Session session) node(TableWriterNode.class, exchange(LOCAL, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), - exchange(LOCAL, REPARTITION, - values("column1", "column2"))) + values("column1", "column2")) .with(exchangeWithoutSystemPartitioning())) .with(exchangeWithoutSystemPartitioning())))); } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java index b5c4671fc165..ef5d17a2ddf9 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLogicalPlanner.java @@ -224,11 +224,10 @@ public void testAggregation() FINAL, exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - aggregation( - ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), - PARTIAL, - anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice")))))))))); + aggregation( + ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), + PARTIAL, + anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice"))))))))); // simple group by over filter that keeps at most one group assertDistributedPlan("SELECT orderstatus, sum(totalprice) FROM orders WHERE orderstatus='O' GROUP BY orderstatus", @@ -238,11 +237,10 @@ public void testAggregation() FINAL, exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - aggregation( - ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), - PARTIAL, - anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice")))))))))); + aggregation( + ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), + PARTIAL, + anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice"))))))))); } @Test @@ -1253,8 +1251,7 @@ public void testUsesDistributedJoinIfNaturallyPartitionedOnProbeSymbols() .right( anyTree( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - tableScan("region", ImmutableMap.of("RIGHT_REGIONKEY", "regionkey")))))))), + tableScan("region", ImmutableMap.of("RIGHT_REGIONKEY", "regionkey"))))))), plan -> // make sure there are only two remote exchanges (one in probe and one in build side) assertEquals( countOfMatchingNodes( @@ -1752,13 +1749,11 @@ public void testRedundantHashRemovalForUnionAllAndMarkDistinct() node(MarkDistinctNode.class, exchange(LOCAL, REPARTITION, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project(ImmutableMap.of("hash_custkey", expression("combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(custkey), 0))"), "hash_nationkey", expression("combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(nationkey), 0))")), - tableScan("customer", ImmutableMap.of("custkey", "custkey", "nationkey", "nationkey"))))), + project(ImmutableMap.of("hash_custkey", expression("combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(custkey), 0))"), "hash_nationkey", expression("combine_hash(bigint '0', COALESCE(\"$operator$hash_code\"(nationkey), 0))")), + tableScan("customer", ImmutableMap.of("custkey", "custkey", "nationkey", "nationkey")))), exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - node(ProjectNode.class, - node(TableScanNode.class))))))))))); + node(ProjectNode.class, + node(TableScanNode.class)))))))))); } @Test @@ -1935,16 +1930,13 @@ public void testGroupingSetsWithDefaultValue() exchange( REMOTE, REPARTITION, - exchange( - LOCAL, - REPARTITION, - aggregation( - ImmutableMap.of("partial_count", functionCall("count", ImmutableList.of("CONSTANT"))), - PARTIAL, - anyTree( - project( - ImmutableMap.of("CONSTANT", expression("1")), - tableScan("orders"))))))))))); + aggregation( + ImmutableMap.of("partial_count", functionCall("count", ImmutableList.of("CONSTANT"))), + PARTIAL, + anyTree( + project( + ImmutableMap.of("CONSTANT", expression("1")), + tableScan("orders")))))))))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java index 9b2ca0855b95..7eddc23d01fa 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestTableScanNodePartitioning.java @@ -187,10 +187,9 @@ void assertTableScanPlannedWithoutPartitioning(Session session, String table) aggregation(ImmutableMap.of("COUNT", functionCall("count", ImmutableList.of("COUNT_PART"))), FINAL, exchange(LOCAL, REPARTITION, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - aggregation(ImmutableMap.of("COUNT_PART", functionCall("count", ImmutableList.of("B"))), PARTIAL, - tableScan(table, ImmutableMap.of("A", "column_a", "B", "column_b")))))))))); + project( + aggregation(ImmutableMap.of("COUNT_PART", functionCall("count", ImmutableList.of("B"))), PARTIAL, + tableScan(table, ImmutableMap.of("A", "column_a", "B", "column_b"))))))))); SubPlan subPlan = subplan(query, OPTIMIZED_AND_VALIDATED, false, session); assertThat(subPlan.getAllFragments()).hasSize(2); assertThat(subPlan.getAllFragments().get(1).getPartitioning().getConnectorHandle()).isEqualTo(SOURCE_DISTRIBUTION.getConnectorHandle()); diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesPlans.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesPlans.java index 9b310bdc6d76..1418e9943d28 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesPlans.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddExchangesPlans.java @@ -167,15 +167,13 @@ public void testRepartitionForUnionAllBeforeHashJoin() anyTree( tableScan("nation", ImmutableMap.of("nationkey", "nationkey")))), exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1"))))))))) + project( + values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1")))))))) .right( anyTree( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - anyTree( - tableScan("region", ImmutableMap.of("regionkey", "regionkey")))))))))); + anyTree( + tableScan("region", ImmutableMap.of("regionkey", "regionkey"))))))))); } @Test @@ -231,17 +229,15 @@ public void testForcePartitioningMarkDistinctInput() node(MarkDistinctNode.class, anyTree( exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition1", "partition2"), - exchange(LOCAL, REPARTITION, - project( - values( - ImmutableList.of("field", "partition2", "partition1"), - ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1"))))))), + project( + values( + ImmutableList.of("field", "partition2", "partition1"), + ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1")))))), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition3", "partition3"), - exchange(LOCAL, REPARTITION, - project( - values( - ImmutableList.of("partition3", "partition4", "field_0"), - ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1"))))))))))); + project( + values( + ImmutableList.of("partition3", "partition4", "field_0"), + ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1")))))))))); assertDistributedPlan( query, @@ -252,17 +248,15 @@ public void testForcePartitioningMarkDistinctInput() node(MarkDistinctNode.class, anyTree( exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition1"), - exchange(LOCAL, REPARTITION, - project( - values( - ImmutableList.of("field", "partition2", "partition1"), - ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1"))))))), + project( + values( + ImmutableList.of("field", "partition2", "partition1"), + ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1")))))), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition3"), - exchange(LOCAL, REPARTITION, - project( - values( - ImmutableList.of("partition3", "partition4", "field_0"), - ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1"))))))))))); + project( + values( + ImmutableList.of("partition3", "partition4", "field_0"), + ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1")))))))))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java index ac910095e93a..36e1e33dea18 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestAddLocalExchangesForTaskScaleWriters.java @@ -168,8 +168,7 @@ public void testLocalScaledPartitionedWriterWithoutSupportForMultipleWritersPerP ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); } @Test(dataProvider = "taskScaleWritersOption") @@ -216,9 +215,8 @@ public void testLocalScaledPartitionedWriterWithoutSupportsForReportingWrittenBy project( exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); assertDistributedPlan( "INSERT INTO connector_partitioned_table SELECT * FROM source_table", @@ -234,8 +232,7 @@ public void testLocalScaledPartitionedWriterWithoutSupportsForReportingWrittenBy ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); } @DataProvider @@ -262,9 +259,8 @@ public void testLocalScaledPartitionedWriterForSystemPartitioning() project( exchange(LOCAL, REPARTITION, SCALED_WRITER_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", @@ -281,9 +277,8 @@ public void testLocalScaledPartitionedWriterForSystemPartitioning() project( exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); } @Test @@ -314,8 +309,7 @@ public void testLocalScaledPartitionedWriterForConnectorPartitioning() ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, scaledPartitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); assertDistributedPlan( "INSERT INTO connector_partitioned_table SELECT * FROM source_table", @@ -331,7 +325,6 @@ public void testLocalScaledPartitionedWriterForConnectorPartitioning() ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestFullOuterJoinWithCoalesce.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestFullOuterJoinWithCoalesce.java index 4c7d84a918f2..2d934be5b4e8 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestFullOuterJoinWithCoalesce.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestFullOuterJoinWithCoalesce.java @@ -125,22 +125,21 @@ public void testCoalesceWithManyArguments() exchange( REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - aggregation( - ImmutableMap.of(), - PARTIAL, - anyTree( - project( - ImmutableMap.of("expr", expression("coalesce(l, m, r)")), - join(FULL, builder -> builder - .equiCriteria("l", "r") - .left( - anyTree( - join(FULL, leftJoinBuilder -> leftJoinBuilder - .equiCriteria("l", "m") - .left(anyTree(values(ImmutableList.of("l")))) - .right(anyTree(values(ImmutableList.of("m"))))))) - .right(anyTree(values(ImmutableList.of("r")))))))))))); + aggregation( + ImmutableMap.of(), + PARTIAL, + anyTree( + project( + ImmutableMap.of("expr", expression("coalesce(l, m, r)")), + join(FULL, builder -> builder + .equiCriteria("l", "r") + .left( + anyTree( + join(FULL, leftJoinBuilder -> leftJoinBuilder + .equiCriteria("l", "m") + .left(anyTree(values(ImmutableList.of("l")))) + .right(anyTree(values(ImmutableList.of("m"))))))) + .right(anyTree(values(ImmutableList.of("r"))))))))))); } @Test @@ -156,21 +155,20 @@ public void testComplexArgumentToCoalesce() exchange( REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - aggregation( - ImmutableMap.of(), - PARTIAL, - anyTree( - project( - ImmutableMap.of("expr", expression("coalesce(l, m + 1, r)")), - join(FULL, builder -> builder - .equiCriteria("l", "r") - .left( - anyTree( - join(FULL, leftJoinBuilder -> leftJoinBuilder - .equiCriteria("l", "m") - .left(anyTree(values(ImmutableList.of("l")))) - .right(anyTree(values(ImmutableList.of("m"))))))) - .right(anyTree(values(ImmutableList.of("r")))))))))))); + aggregation( + ImmutableMap.of(), + PARTIAL, + anyTree( + project( + ImmutableMap.of("expr", expression("coalesce(l, m + 1, r)")), + join(FULL, builder -> builder + .equiCriteria("l", "r") + .left( + anyTree( + join(FULL, leftJoinBuilder -> leftJoinBuilder + .equiCriteria("l", "m") + .left(anyTree(values(ImmutableList.of("l")))) + .right(anyTree(values(ImmutableList.of("m"))))))) + .right(anyTree(values(ImmutableList.of("r"))))))))))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestWindow.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestWindow.java index d8a6a5af462e..d618a3d68f4a 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestWindow.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestWindow.java @@ -85,8 +85,7 @@ public void testWindow() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus"))))))))); + project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus")))))))); assertDistributedPlan("SELECT row_number() OVER (PARTITION BY orderstatus) FROM orders", anyTree( @@ -94,8 +93,7 @@ public void testWindow() .partitionBy(ImmutableList.of("orderstatus")), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus"))))))))); + project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus")))))))); assertDistributedPlan("SELECT orderstatus FROM (SELECT orderstatus, row_number() OVER (PARTITION BY orderstatus ORDER BY custkey) n FROM orders) WHERE n = 1", anyTree( @@ -107,14 +105,13 @@ public void testWindow() .partial(false), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - topNRanking(topNRanking -> topNRanking - .specification( - ImmutableList.of("orderstatus"), - ImmutableList.of("custkey"), - ImmutableMap.of("custkey", ASC_NULLS_LAST)) - .partial(true), - project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus", "custkey", "custkey")))))))))); + topNRanking(topNRanking -> topNRanking + .specification( + ImmutableList.of("orderstatus"), + ImmutableList.of("custkey"), + ImmutableMap.of("custkey", ASC_NULLS_LAST)) + .partial(true), + project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus", "custkey", "custkey"))))))))); } @Test @@ -178,17 +175,16 @@ public void testWindowAfterJoin() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - join(INNER, builder -> builder - .equiCriteria("orderstatus", "linestatus") - .distributionType(REPLICATED) - .left( - anyTree(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus", "custkey", "custkey")))) - .right( - exchange(LOCAL, GATHER, - exchange(REMOTE, REPLICATE, - anyTree(tableScan("lineitem", ImmutableMap.of("linestatus", "linestatus")))))))))))))); + project( + join(INNER, builder -> builder + .equiCriteria("orderstatus", "linestatus") + .distributionType(REPLICATED) + .left( + anyTree(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus", "custkey", "custkey")))) + .right( + exchange(LOCAL, GATHER, + exchange(REMOTE, REPLICATE, + anyTree(tableScan("lineitem", ImmutableMap.of("linestatus", "linestatus"))))))))))))); } @Test @@ -203,8 +199,7 @@ public void testWindowAfterAggregation() project(aggregation(singleGroupingSet("custkey"), ImmutableMap.of(), Optional.empty(), FINAL, exchange(LOCAL, GATHER, project(exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - anyTree(tableScan("orders", ImmutableMap.of("custkey", "custkey")))))))))))); + anyTree(tableScan("orders", ImmutableMap.of("custkey", "custkey"))))))))))); // Window partition key is not a super set of group by key. assertDistributedPlan("SELECT rank() OVER (partition by custkey) FROM (SELECT shippriority, custkey, sum(totalprice) FROM orders GROUP BY shippriority, custkey)", @@ -214,11 +209,9 @@ public void testWindowAfterAggregation() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project(aggregation(singleGroupingSet("shippriority", "custkey"), ImmutableMap.of(), Optional.empty(), FINAL, - exchange(LOCAL, GATHER, - exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - anyTree(tableScan("orders", ImmutableMap.of("custkey", "custkey", "shippriority", "shippriority")))))))))))))); + project(aggregation(singleGroupingSet("shippriority", "custkey"), ImmutableMap.of(), Optional.empty(), FINAL, + exchange(LOCAL, GATHER, + exchange(REMOTE, REPARTITION, + anyTree(tableScan("orders", ImmutableMap.of("custkey", "custkey", "shippriority", "shippriority")))))))))))); } } diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java index bd2ae04e6c36..7d087b7c81b9 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/BaseJdbcConnectorTest.java @@ -452,13 +452,13 @@ public void testDistinctAggregationPushdown() withMarkDistinct, "SELECT count(DISTINCT regionkey), sum(nationkey) FROM nation", hasBehavior(SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT), - node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))))))); + node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class)))))); // distinct aggregation and a non-distinct aggregation assertConditionallyPushedDown( withMarkDistinct, "SELECT count(DISTINCT regionkey), count(DISTINCT nationkey) FROM nation", hasBehavior(SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT), - node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))))))); + node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class)))))); Session withoutMarkDistinct = Session.builder(getSession()) .setSystemProperty(USE_MARK_DISTINCT, "false") @@ -563,7 +563,7 @@ public void testCountDistinctWithStringTypes() assertThat(query("SELECT count(DISTINCT t_char), count(DISTINCT t_varchar) FROM " + testTable.getName())) .matches("VALUES (BIGINT '7', BIGINT '7')") - .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); } else { // Single count(DISTINCT ...) can be pushed even down even if SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT == false as GROUP BY @@ -580,7 +580,7 @@ public void testCountDistinctWithStringTypes() getSession(), "SELECT count(DISTINCT t_char), count(DISTINCT t_varchar) FROM " + testTable.getName(), hasBehavior(SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT), - node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))))))); + node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class)))))); } } } @@ -974,7 +974,7 @@ public void testTopNPushdown() .isNotFullyPushedDown( node(TopNNode.class, // FINAL TopN anyTree(node(JoinNode.class, - node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class)))), // no PARTIAL TopN + node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))), // no PARTIAL TopN anyTree(node(TableScanNode.class)))))); } @@ -1127,6 +1127,7 @@ public void testJoinPushdownDisabled() // Disable optimized hash generation so that expected plans in case of no pushdown remain "simple" .setSystemProperty("optimize_hash_generation", "false") .build(); + assertThat(query(noJoinPushdown, "SELECT r.name, n.name FROM nation n JOIN region r ON n.regionkey = r.regionkey")) .joinIsNotFullyPushedDown(); } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index f40dc5085425..1223edc6bdf7 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -2163,7 +2163,7 @@ private void testBucketedTable(Session session, HiveStorageFormat storageFormat, "INSERT INTO " + tableName + " VALUES ('a0', 'b0', 'c0')", 1, // buckets should be repartitioned locally hence local repartitioned exchange should exist in plan - assertLocalRepartitionedExchangesCount(2)); + assertLocalRepartitionedExchangesCount(1)); assertUpdate(parallelWriter, "INSERT INTO " + tableName + " VALUES ('a1', 'b1', 'c1')", 1); assertQuery("SELECT * from " + tableName, "VALUES ('a', 'b', 'c'), ('aa', 'bb', 'cc'), ('aaa', 'bbb', 'ccc'), ('a0', 'b0', 'c0'), ('a1', 'b1', 'c1')"); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java index 6b64eff239f4..9588285a3aaa 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/optimizer/TestHivePlans.java @@ -170,16 +170,14 @@ public void testPrunePartitionLikeFilter() .equiCriteria("L_STR_PART", "R_STR_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("\"$like\"(L_STR_PART, \"$literal$\"(from_base64('DgAAAFZBUklBQkxFX1dJRFRIAQAAAAcAAAAABwAAAAIAAAB0JQA=')))", - tableScan("table_str_partitioned", Map.of("L_INT_COL", "int_col", "L_STR_PART", "str_part"))))))) + project( + filter("\"$like\"(L_STR_PART, \"$literal$\"(from_base64('DgAAAFZBUklBQkxFX1dJRFRIAQAAAAcAAAAABwAAAAIAAAB0JQA=')))", + tableScan("table_str_partitioned", Map.of("L_INT_COL", "int_col", "L_STR_PART", "str_part")))))) .right(exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_STR_COL IN ('three', CAST('two' AS varchar(5))) AND \"$like\"(R_STR_COL, \"$literal$\"(from_base64('DgAAAFZBUklBQkxFX1dJRFRIAQAAAAcAAAAABwAAAAIAAAB0JQA=')))", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_STR_COL IN ('three', CAST('two' AS varchar(5))) AND \"$like\"(R_STR_COL, \"$literal$\"(from_base64('DgAAAFZBUklBQkxFX1dJRFRIAQAAAAcAAAAABwAAAAIAAAB0JQA=')))", + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test @@ -196,17 +194,15 @@ public void testSubsumePartitionFilter() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("true", // dynamic filter - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col"))))))) + project( + filter("true", // dynamic filter + tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) .right( exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_INT_COL IN (2, 3, 4)", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_INT_COL IN (2, 3, 4)", + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test @@ -224,17 +220,15 @@ public void testSubsumePartitionPartOfAFilter() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("L_STR_COL != 'three'", - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col"))))))) + project( + filter("L_STR_COL != 'three'", + tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) .right( exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_INT_COL IN (2, 3, 4) AND R_INT_COL BETWEEN 2 AND 4", // TODO: R_INT_COL BETWEEN 2 AND 4 is redundant - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_INT_COL IN (2, 3, 4) AND R_INT_COL BETWEEN 2 AND 4", // TODO: R_INT_COL BETWEEN 2 AND 4 is redundant + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test @@ -252,17 +246,15 @@ public void testSubsumePartitionPartWhenOtherFilterNotConvertibleToTupleDomain() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("substring(L_STR_COL, BIGINT '2') != CAST('hree' AS varchar(5))", - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col"))))))) + project( + filter("substring(L_STR_COL, BIGINT '2') != CAST('hree' AS varchar(5))", + tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) .right( exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_INT_COL IN (2, 3, 4) AND R_INT_COL BETWEEN 2 AND 4", // TODO: R_INT_COL BETWEEN 2 AND 4 is redundant - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_INT_COL IN (2, 3, 4) AND R_INT_COL BETWEEN 2 AND 4", // TODO: R_INT_COL BETWEEN 2 AND 4 is redundant + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test @@ -280,17 +272,15 @@ public void testSubsumePartitionFilterNotConvertibleToTupleDomain() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("L_INT_PART % 2 = 0", - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col"))))))) + project( + filter("L_INT_PART % 2 = 0", + tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) .right( exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_INT_COL IN (2, 4) AND R_INT_COL % 2 = 0", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_INT_COL IN (2, 4) AND R_INT_COL % 2 = 0", + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test @@ -305,17 +295,15 @@ public void testFilterDerivedFromTableProperties() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("true", //dynamic filter - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col"))))))) + project( + filter("true", //dynamic filter + tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) .right( exchange(LOCAL, exchange(REMOTE, REPARTITION, - exchange(LOCAL, REPARTITION, - project( - filter("R_INT_COL IN (1, 2, 3, 4, 5)", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); + project( + filter("R_INT_COL IN (1, 2, 3, 4, 5)", + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); } @Test diff --git a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlLegacyConnectorTest.java b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlLegacyConnectorTest.java index eba30762e7b8..491d988def4d 100644 --- a/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlLegacyConnectorTest.java +++ b/plugin/trino-mysql/src/test/java/io/trino/plugin/mysql/TestMySqlLegacyConnectorTest.java @@ -122,7 +122,7 @@ public void testCountDistinctWithStringTypes() assertThat(query("SELECT count(DISTINCT t_char), count(DISTINCT t_varchar) FROM " + testTable.getName())) .matches("VALUES (BIGINT '7', BIGINT '7')") - .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); } } diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotIntegrationConnectorSmokeTest.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotIntegrationConnectorSmokeTest.java index f14bb1c01a51..62194425ed08 100644 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotIntegrationConnectorSmokeTest.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/BasePinotIntegrationConnectorSmokeTest.java @@ -1702,11 +1702,11 @@ public void testAggregationPushdown() // Passthrough queries with aggregates will not push down more aggregations. assertThat(query("SELECT bool_col, \"count(*)\", COUNT(*) FROM \"SELECT bool_col, count(*) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col\" GROUP BY bool_col, \"count(*)\"")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); assertThat(query("SELECT bool_col, \"max(long_col)\", COUNT(*) FROM \"SELECT bool_col, max(long_col) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col\" GROUP BY bool_col, \"max(long_col)\"")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); assertThat(query("SELECT int_col, COUNT(*) FROM " + ALL_TYPES_TABLE + " GROUP BY int_col LIMIT " + MAX_ROWS_PER_SPLIT_FOR_SEGMENT_QUERIES)) .isFullyPushedDown(); @@ -1714,30 +1714,30 @@ public void testAggregationPushdown() // count() should not be pushed down, as pinot currently only implements count(*) assertThat(query("SELECT bool_col, COUNT(long_col)" + " FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // AVG on INTEGER columns is not pushed down assertThat(query("SELECT string_col, AVG(int_col) FROM " + ALL_TYPES_TABLE + " GROUP BY string_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // SUM on INTEGER columns is not pushed down assertThat(query("SELECT string_col, SUM(int_col) FROM " + ALL_TYPES_TABLE + " GROUP BY string_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // MIN on VARCHAR columns is not pushed down assertThat(query("SELECT bool_col, MIN(string_col)" + " FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // MAX on VARCHAR columns is not pushed down assertThat(query("SELECT bool_col, MAX(string_col)" + " FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // COUNT on VARCHAR columns is not pushed down assertThat(query("SELECT bool_col, COUNT(string_col)" + " FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // Distinct on varchar is pushed down assertThat(query("SELECT DISTINCT string_col FROM " + ALL_TYPES_TABLE)) @@ -1845,22 +1845,22 @@ public void testAggregationPushdown() // Aggregation is not pushed down for queries with count distinct and other aggregations assertThat(query("SELECT bool_col, MAX(long_col), COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT bool_col, COUNT(DISTINCT long_col), MAX(long_col) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT bool_col, COUNT(*), COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT bool_col, COUNT(DISTINCT long_col), COUNT(*) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); // Test queries with no grouping columns assertThat(query("SELECT MAX(long_col), COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE)) - .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT COUNT(DISTINCT long_col), MAX(long_col) FROM " + ALL_TYPES_TABLE)) - .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT COUNT(*), COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE)) - .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); assertThat(query("SELECT COUNT(DISTINCT long_col), COUNT(*) FROM " + ALL_TYPES_TABLE)) - .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class); Session countDistinctPushdownDisabledSession = Session.builder(getQueryRunner().getDefaultSession()) .setCatalogSessionProperty("pinot", "count_distinct_pushdown_enabled", "false") @@ -1868,7 +1868,7 @@ public void testAggregationPushdown() // Distinct count is partially pushed down when the distinct_count_pushdown_enabled session property is disabled assertThat(query(countDistinctPushdownDisabledSession, "SELECT bool_col, COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE + " GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // Test query with no grouping columns assertThat(query(countDistinctPushdownDisabledSession, "SELECT COUNT(DISTINCT long_col) FROM " + ALL_TYPES_TABLE)) .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class); @@ -1881,7 +1881,7 @@ public void testAggregationPushdown() // Ensure that count() is not pushed down even when a broker query is present and has grouping columns // This is also done as the second step of count distinct but should not be pushed down in this case. assertThat(query("SELECT bool_col, COUNT(long_col) FROM \"SELECT bool_col, long_col FROM " + ALL_TYPES_TABLE + "\" GROUP BY bool_col")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // Ensure that count() is not pushed down even if the query contains a matching grouping column assertThatExceptionOfType(RuntimeException.class) @@ -2298,7 +2298,7 @@ public void testLimitAndOffsetWithPushedDownAggregates() " (BIGINT '-3147483640', BIGINT '1', BIGINT '-3147483640')," + " (BIGINT '-3147483641', BIGINT '1', BIGINT '-3147483641')," + " (BIGINT '-3147483639', BIGINT '1', BIGINT '-3147483639')") - .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class, AggregationNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ProjectNode.class, AggregationNode.class); assertThat(query("SELECT long_col, string_col, COUNT(*), MAX(long_col)" + " FROM \"SELECT * FROM " + ALL_TYPES_TABLE + @@ -2309,7 +2309,7 @@ public void testLimitAndOffsetWithPushedDownAggregates() " (BIGINT '-3147483640', VARCHAR 'string_8400', BIGINT '1', BIGINT '-3147483640')," + " (BIGINT '-3147483642', VARCHAR 'string_6000', BIGINT '1', BIGINT '-3147483642')," + " (BIGINT '-3147483639', VARCHAR 'string_9600', BIGINT '1', BIGINT '-3147483639')") - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); // Note that the offset is the first parameter assertThat(query("SELECT long_col" + @@ -2400,9 +2400,9 @@ public void testAggregatePassthroughQueriesWithExpressions() public void testAggregationPushdownWithArrays() { assertThat(query("SELECT string_array_col, count(*) FROM " + ALL_TYPES_TABLE + " WHERE int_col = 54 GROUP BY 1")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); assertThat(query("SELECT int_array_col, string_array_col, count(*) FROM " + ALL_TYPES_TABLE + " WHERE int_col = 54 GROUP BY 1, 2")) - .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, ProjectNode.class); assertThat(query("SELECT int_array_col, \"count(*)\"" + " FROM \"SELECT int_array_col, COUNT(*) FROM " + ALL_TYPES_TABLE + " WHERE int_col = 54 GROUP BY 1\"")) diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java index 603d54f666d6..f9c3d8b58a05 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/TestFaultTolerantExecutionDynamicFiltering.java @@ -56,9 +56,8 @@ protected QueryRunner createQueryRunner() }) .setExtraProperties(FaultTolerantExecutionConnectorTestHelper.getExtraProperties()) // keep limits lower to test edge cases - .addExtraProperty("dynamic-filtering.small-partitioned.max-distinct-values-per-driver", "20") - .addExtraProperty("dynamic-filtering.small-broadcast.max-distinct-values-per-driver", "20") - .addExtraProperty("dynamic-filtering.small-partitioned.range-row-limit-per-driver", "200") + .addExtraProperty("dynamic-filtering.small-partitioned.max-distinct-values-per-driver", "10") + .addExtraProperty("dynamic-filtering.small-broadcast.max-distinct-values-per-driver", "10") .build(); } diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q01.plan.txt index 4505e1d574d5..61a20a28c4d3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q01.plan.txt @@ -10,13 +10,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store @@ -25,18 +24,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sr_store_sk_15) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_15) - final aggregation over (sr_customer_sk_11, sr_store_sk_15) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_11", "sr_store_sk_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_11, sr_store_sk_15) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_15) + final aggregation over (sr_customer_sk_11, sr_store_sk_15) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_11", "sr_store_sk_15"]) + partial aggregation over (sr_customer_sk_11, sr_store_sk_15) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q02.plan.txt index 522307dea3c5..7605dbd6e9ad 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_229"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_132) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_132"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_132) - final aggregation over (d_day_name_142, d_week_seq_132) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_142", "d_week_seq_132"]) - partial aggregation over (d_day_name_142, d_week_seq_132) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_132) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_178"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_132"]) + partial aggregation over (d_week_seq_132) + final aggregation over (d_day_name_142, d_week_seq_132) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_142", "d_week_seq_132"]) + partial aggregation over (d_day_name_142, d_week_seq_132) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_178"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q04.plan.txt index adf59a0d1a7c..99aa88843f21 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q04.plan.txt @@ -3,118 +3,100 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_574"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_587, c_customer_id_574, c_email_address_589, c_first_name_581, c_last_name_582, c_login_588, c_preferred_cust_flag_583, d_year_638) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_587", "c_customer_id_574", "c_email_address_589", "c_first_name_581", "c_last_name_582", "c_login_588", "c_preferred_cust_flag_583", "d_year_638"]) - partial aggregation over (c_birth_country_587, c_customer_id_574, c_email_address_589, c_first_name_581, c_last_name_582, c_login_588, c_preferred_cust_flag_583, d_year_638) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_573"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (c_birth_country_587, c_customer_id_574, c_email_address_589, c_first_name_581, c_last_name_582, c_login_588, c_preferred_cust_flag_583, d_year_638) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_587", "c_customer_id_574", "c_email_address_589", "c_first_name_581", "c_last_name_582", "c_login_588", "c_preferred_cust_flag_583", "d_year_638"]) + partial aggregation over (c_birth_country_587, c_customer_id_574, c_email_address_589, c_first_name_581, c_last_name_582, c_login_588, c_preferred_cust_flag_583, d_year_638) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_573"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_596"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_1611"]) + final aggregation over (c_birth_country_1624, c_customer_id_1611, c_email_address_1626, c_first_name_1618, c_last_name_1619, c_login_1625, c_preferred_cust_flag_1620, d_year_1675) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1624", "c_customer_id_1611", "c_email_address_1626", "c_first_name_1618", "c_last_name_1619", "c_login_1625", "c_preferred_cust_flag_1620", "d_year_1675"]) + partial aggregation over (c_birth_country_1624, c_customer_id_1611, c_email_address_1626, c_first_name_1618, c_last_name_1619, c_login_1625, c_preferred_cust_flag_1620, d_year_1675) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_1610"]) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_596"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1634"]) join (INNER, REPLICATED): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_1611"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1624, c_customer_id_1611, c_email_address_1626, c_first_name_1618, c_last_name_1619, c_login_1625, c_preferred_cust_flag_1620, d_year_1675) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1299"]) + final aggregation over (c_birth_country_1312, c_customer_id_1299, c_email_address_1314, c_first_name_1306, c_last_name_1307, c_login_1313, c_preferred_cust_flag_1308, d_year_1363) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1624", "c_customer_id_1611", "c_email_address_1626", "c_first_name_1618", "c_last_name_1619", "c_login_1625", "c_preferred_cust_flag_1620", "d_year_1675"]) - partial aggregation over (c_birth_country_1624, c_customer_id_1611, c_email_address_1626, c_first_name_1618, c_last_name_1619, c_login_1625, c_preferred_cust_flag_1620, d_year_1675) + remote exchange (REPARTITION, HASH, ["c_birth_country_1312", "c_customer_id_1299", "c_email_address_1314", "c_first_name_1306", "c_last_name_1307", "c_login_1313", "c_preferred_cust_flag_1308", "d_year_1363"]) + partial aggregation over (c_birth_country_1312, c_customer_id_1299, c_email_address_1314, c_first_name_1306, c_last_name_1307, c_login_1313, c_preferred_cust_flag_1308, d_year_1363) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_1610"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1298"]) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1634"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1299"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1312, c_customer_id_1299, c_email_address_1314, c_first_name_1306, c_last_name_1307, c_login_1313, c_preferred_cust_flag_1308, d_year_1363) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1312", "c_customer_id_1299", "c_email_address_1314", "c_first_name_1306", "c_last_name_1307", "c_login_1313", "c_preferred_cust_flag_1308", "d_year_1363"]) - partial aggregation over (c_birth_country_1312, c_customer_id_1299, c_email_address_1314, c_first_name_1306, c_last_name_1307, c_login_1313, c_preferred_cust_flag_1308, d_year_1363) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_1298"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1322"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1322"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id_886"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_899, c_customer_id_886, c_email_address_901, c_first_name_893, c_last_name_894, c_login_900, c_preferred_cust_flag_895, d_year_950) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_899", "c_customer_id_886", "c_email_address_901", "c_first_name_893", "c_last_name_894", "c_login_900", "c_preferred_cust_flag_895", "d_year_950"]) - partial aggregation over (c_birth_country_899, c_customer_id_886, c_email_address_901, c_first_name_893, c_last_name_894, c_login_900, c_preferred_cust_flag_895, d_year_950) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_885"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_908"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (c_birth_country_899, c_customer_id_886, c_email_address_901, c_first_name_893, c_last_name_894, c_login_900, c_preferred_cust_flag_895, d_year_950) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_899", "c_customer_id_886", "c_email_address_901", "c_first_name_893", "c_last_name_894", "c_login_900", "c_preferred_cust_flag_895", "d_year_950"]) + partial aggregation over (c_birth_country_899, c_customer_id_886, c_email_address_901, c_first_name_893, c_last_name_894, c_login_900, c_preferred_cust_flag_895, d_year_950) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_885"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_908"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id_172"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_185, c_customer_id_172, c_email_address_187, c_first_name_179, c_last_name_180, c_login_186, c_preferred_cust_flag_181, d_year_225) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_185", "c_customer_id_172", "c_email_address_187", "c_first_name_179", "c_last_name_180", "c_login_186", "c_preferred_cust_flag_181", "d_year_225"]) - partial aggregation over (c_birth_country_185, c_customer_id_172, c_email_address_187, c_first_name_179, c_last_name_180, c_login_186, c_preferred_cust_flag_181, d_year_225) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_194"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_171"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country_185, c_customer_id_172, c_email_address_187, c_first_name_179, c_last_name_180, c_login_186, c_preferred_cust_flag_181, d_year_225) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_185", "c_customer_id_172", "c_email_address_187", "c_first_name_179", "c_last_name_180", "c_login_186", "c_preferred_cust_flag_181", "d_year_225"]) + partial aggregation over (c_birth_country_185, c_customer_id_172, c_email_address_187, c_first_name_179, c_last_name_180, c_login_186, c_preferred_cust_flag_181, d_year_225) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_194"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_171"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q05.plan.txt index a3e45e69889a..f98eebbfad6e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_96", "ws_order_number_110"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q06.plan.txt index 78c06d9102ba..d3d2cfba574a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q06.plan.txt @@ -10,45 +10,39 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_15) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_15) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_15) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_15"]) + partial aggregation over (d_month_seq_15) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (i_category_55) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_55"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_55) - scan item + partial aggregation over (i_category_55) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q07.plan.txt index c13e8aa2f577..cbab34a925c5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q07.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q08.plan.txt index b833e024e439..432e37f5b19c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_39"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_40"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_31) - scan customer_address + partial aggregation over (ca_zip_31) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_34) - final aggregation over (ca_zip_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_19"]) - partial aggregation over (ca_zip_19) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_34) + final aggregation over (ca_zip_19) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_19"]) + partial aggregation over (ca_zip_19) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_10"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q10.plan.txt index 74c23aa40452..1cee500dbb4b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q10.plan.txt @@ -3,53 +3,47 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q11.plan.txt index 9c95b298e45d..3e8cc6b9923a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q11.plan.txt @@ -2,79 +2,67 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_85"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_98, c_customer_id_85, c_email_address_100, c_first_name_92, c_last_name_93, c_login_99, c_preferred_cust_flag_94, d_year_138) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_98", "c_customer_id_85", "c_email_address_100", "c_first_name_92", "c_last_name_93", "c_login_99", "c_preferred_cust_flag_94", "d_year_138"]) - partial aggregation over (c_birth_country_98, c_customer_id_85, c_email_address_100, c_first_name_92, c_last_name_93, c_login_99, c_preferred_cust_flag_94, d_year_138) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (c_birth_country_98, c_customer_id_85, c_email_address_100, c_first_name_92, c_last_name_93, c_login_99, c_preferred_cust_flag_94, d_year_138) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_98", "c_customer_id_85", "c_email_address_100", "c_first_name_92", "c_last_name_93", "c_login_99", "c_preferred_cust_flag_94", "d_year_138"]) + partial aggregation over (c_birth_country_98, c_customer_id_85, c_email_address_100, c_first_name_92, c_last_name_93, c_login_99, c_preferred_cust_flag_94, d_year_138) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_84"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk_107"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id"]) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk_107"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) join (INNER, REPLICATED): scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_577"]) + final aggregation over (c_birth_country_590, c_customer_id_577, c_email_address_592, c_first_name_584, c_last_name_585, c_login_591, c_preferred_cust_flag_586, d_year_641) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_590", "c_customer_id_577", "c_email_address_592", "c_first_name_584", "c_last_name_585", "c_login_591", "c_preferred_cust_flag_586", "d_year_641"]) + partial aggregation over (c_birth_country_590, c_customer_id_577, c_email_address_592, c_first_name_584, c_last_name_585, c_login_591, c_preferred_cust_flag_586, d_year_641) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_576"]) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_600"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_577"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_590, c_customer_id_577, c_email_address_592, c_first_name_584, c_last_name_585, c_login_591, c_preferred_cust_flag_586, d_year_641) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_376"]) + final aggregation over (c_birth_country_389, c_customer_id_376, c_email_address_391, c_first_name_383, c_last_name_384, c_login_390, c_preferred_cust_flag_385, d_year_440) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_590", "c_customer_id_577", "c_email_address_592", "c_first_name_584", "c_last_name_585", "c_login_591", "c_preferred_cust_flag_586", "d_year_641"]) - partial aggregation over (c_birth_country_590, c_customer_id_577, c_email_address_592, c_first_name_584, c_last_name_585, c_login_591, c_preferred_cust_flag_586, d_year_641) + remote exchange (REPARTITION, HASH, ["c_birth_country_389", "c_customer_id_376", "c_email_address_391", "c_first_name_383", "c_last_name_384", "c_login_390", "c_preferred_cust_flag_385", "d_year_440"]) + partial aggregation over (c_birth_country_389, c_customer_id_376, c_email_address_391, c_first_name_383, c_last_name_384, c_login_390, c_preferred_cust_flag_385, d_year_440) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_576"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_375"]) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_600"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_376"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_389, c_customer_id_376, c_email_address_391, c_first_name_383, c_last_name_384, c_login_390, c_preferred_cust_flag_385, d_year_440) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_389", "c_customer_id_376", "c_email_address_391", "c_first_name_383", "c_last_name_384", "c_login_390", "c_preferred_cust_flag_385", "d_year_440"]) - partial aggregation over (c_birth_country_389, c_customer_id_376, c_email_address_391, c_first_name_383, c_last_name_384, c_login_390, c_preferred_cust_flag_385, d_year_440) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_375"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_399"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_399"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q12.plan.txt index 78a75ae079e8..f7ec908dd084 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q12.plan.txt @@ -2,20 +2,17 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q13.plan.txt index 7a12976d2b07..515bc8719e42 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q13.plan.txt @@ -5,24 +5,22 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q14.plan.txt index c7254869e697..ac9f82f11b05 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_15) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_15"]) - partial aggregation over (i_item_sk_15) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_22", "i_category_id_26", "i_class_id_24"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_77", "i_category_id_81", "i_class_id_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_77, i_category_id_81, i_class_id_79) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_137", "i_category_id_141", "i_class_id_139"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_137, i_category_id_141, i_class_id_139) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_197", "i_category_id_201", "i_class_id_199"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_197, i_category_id_201, i_class_id_199) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_15) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_15"]) + partial aggregation over (i_item_sk_15) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_22", "i_category_id_26", "i_class_id_24"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_77", "i_category_id_81", "i_class_id_79"]) + partial aggregation over (i_brand_id_77, i_category_id_81, i_class_id_79) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_137", "i_category_id_141", "i_class_id_139"]) + partial aggregation over (i_brand_id_137, i_category_id_141, i_class_id_139) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_197", "i_category_id_201", "i_class_id_199"]) + partial aggregation over (i_brand_id_197, i_category_id_201, i_class_id_199) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_495, i_category_id_499, i_class_id_497) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_495", "i_category_id_499", "i_class_id_497"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_495, i_category_id_499, i_class_id_497) + partial aggregation over (i_brand_id_495, i_category_id_499, i_class_id_497) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_488"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_546) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_546"]) - partial aggregation over (i_item_sk_546) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_553", "i_category_id_557", "i_class_id_555"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_571, category_id_573, class_id_572) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_611", "i_category_id_615", "i_class_id_613"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_611, i_category_id_615, i_class_id_613) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_705", "i_category_id_709", "i_class_id_707"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_705, i_category_id_709, i_class_id_707) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_799", "i_category_id_803", "i_class_id_801"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_799, i_category_id_803, i_class_id_801) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_488"]) + scan item + final aggregation over (i_item_sk_546) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_546"]) + partial aggregation over (i_item_sk_546) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_553", "i_category_id_557", "i_class_id_555"]) + scan item + final aggregation over (brand_id_571, category_id_573, class_id_572) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_611", "i_category_id_615", "i_class_id_613"]) + partial aggregation over (i_brand_id_611, i_category_id_615, i_class_id_613) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_705", "i_category_id_709", "i_class_id_707"]) + partial aggregation over (i_brand_id_705, i_category_id_709, i_class_id_707) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_799", "i_category_id_803", "i_class_id_801"]) + partial aggregation over (i_brand_id_799, i_category_id_803, i_class_id_801) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -186,63 +174,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_1104, i_category_id_1108, i_class_id_1106) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_1061"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_1097"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_sk_1155) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_1155"]) partial aggregation over (i_item_sk_1155) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["i_brand_id_1162", "i_category_id_1166", "i_class_id_1164"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (brand_id_1180, category_id_1182, class_id_1181) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1220", "i_category_id_1224", "i_class_id_1222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1220, i_category_id_1224, i_class_id_1222) + partial aggregation over (i_brand_id_1220, i_category_id_1224, i_class_id_1222) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_1314", "i_category_id_1318", "i_class_id_1316"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1314, i_category_id_1318, i_class_id_1316) + partial aggregation over (i_brand_id_1314, i_category_id_1318, i_class_id_1316) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_1408", "i_category_id_1412", "i_class_id_1410"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1408, i_category_id_1412, i_class_id_1410) + partial aggregation over (i_brand_id_1408, i_category_id_1412, i_class_id_1410) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q15.plan.txt index a9f23b963ee8..946683bd17b5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q16.plan.txt index ec35505379b1..2583381afcd8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q16.plan.txt @@ -4,35 +4,32 @@ final aggregation over () partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (LEFT, REPLICATED): - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_25, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_25", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_25, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + join (LEFT, REPLICATED): + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_25, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_25", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_25, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cr_order_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cr_order_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_order_number"]) + partial aggregation over (cr_order_number) + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q17.plan.txt index 402e64a4b7b7..add2e9b8dfad 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q17.plan.txt @@ -3,37 +3,34 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_desc, i_item_id, s_state) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_desc", "i_item_id", "s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_desc, i_item_id, s_state) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_item_desc, i_item_id, s_state) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q18.plan.txt index 79fba25ad0a6..f3ff54aad741 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q18.plan.txt @@ -4,37 +4,34 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) remote exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) partial aggregation over (ca_country$gid, ca_county$gid, ca_state$gid, groupid, i_item_id$gid) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + local exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) + remote exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_3"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + remote exchange (REPARTITION, HASH, ["cd_demo_sk_3"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q19.plan.txt index 98f1c759ed77..43f0630f3559 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q19.plan.txt @@ -3,26 +3,25 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id", "i_manufact", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q20.plan.txt index b2605bcac742..95a4e8334e25 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q20.plan.txt @@ -2,20 +2,17 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q21.plan.txt index 24265297745e..c127db88db46 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q23.plan.txt index d15fdcf05bf2..81cbaf214536 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_9, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_9", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_9, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_9, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_9", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_9, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_47) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_47) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_47) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_47"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_78) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_103"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_199) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_199"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_199) - final aggregation over (d_date_227, ss_item_sk_199, substr$gid_284) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_227", "ss_item_sk_199", "substr$gid_284"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_227, ss_item_sk_199, substr$gid_284) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_199) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_199"]) + partial aggregation over (ss_item_sk_199) + final aggregation over (d_date_227, ss_item_sk_199, substr$gid_284) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_227", "ss_item_sk_199", "substr$gid_284"]) + partial aggregation over (d_date_227, ss_item_sk_199, substr$gid_284) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_292) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_292) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_292) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_292"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_317"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_343) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_343"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_368"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q24.plan.txt index e039646ac696..b24eafd9738a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q24.plan.txt @@ -5,40 +5,35 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -56,24 +51,20 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk_19"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_18", "ss_ticket_number_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_45", "sr_ticket_number_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_125"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_155", "upper_173"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q25.plan.txt index 48b7cc7657a3..06e9bc127b43 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q25.plan.txt @@ -3,40 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_desc", "i_item_id", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_store_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_sk"]) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q26.plan.txt index 959a8e6cf638..10e8a2551020 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q26.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q27.plan.txt index a743f4906e3f..e15f77571f95 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q27.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_item_id$gid, s_state$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q28.plan.txt index 747a5c351502..d917765996a2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_72"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_102"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_132"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q29.plan.txt index 0b0b1066f9fd..e563e2d9b18a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q29.plan.txt @@ -3,37 +3,34 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_desc", "i_item_id", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["s_store_sk"]) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q30.plan.txt index f966fccd867d..120f70ae6d63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_92) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_92"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_92) - final aggregation over (ca_state_92, wr_returning_customer_sk_31) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_92", "wr_returning_customer_sk_31"]) - partial aggregation over (ca_state_92, wr_returning_customer_sk_31) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ca_state_92) + final aggregation over (ca_state_92, wr_returning_customer_sk_31) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_92", "wr_returning_customer_sk_31"]) + partial aggregation over (ca_state_92, wr_returning_customer_sk_31) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_84"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_34"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q31.plan.txt index 442731513c1f..2ae87cf0822b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_71, d_qoy_43, d_year_39) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_64"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_149", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_149, d_qoy_121, d_year_117) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_149", "d_qoy_121", "d_year_117"]) - partial aggregation over (ca_county_149, d_qoy_121, d_year_117) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_89"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_142"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_149, d_qoy_121, d_year_117) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_149", "d_qoy_121", "d_year_117"]) + partial aggregation over (ca_county_149, d_qoy_121, d_year_117) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_89"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_142"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_293, d_qoy_265, d_year_261) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_293, d_qoy_265, d_year_261) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_286"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_223"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_382", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_382, d_qoy_354, d_year_350) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_382", "d_qoy_354", "d_year_350"]) - partial aggregation over (ca_county_382, d_qoy_354, d_year_350) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_375"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_312"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) + final aggregation over (ca_county_382, d_qoy_354, d_year_350) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) + remote exchange (REPARTITION, HASH, ["ca_county_382", "d_qoy_354", "d_year_350"]) + partial aggregation over (ca_county_382, d_qoy_354, d_year_350) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_375"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_312"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_204, d_qoy_176, d_year_172) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_204", "d_qoy_176", "d_year_172"]) - partial aggregation over (ca_county_204, d_qoy_176, d_year_172) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_197"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_204, d_qoy_176, d_year_172) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_204", "d_qoy_176", "d_year_172"]) + partial aggregation over (ca_county_204, d_qoy_176, d_year_172) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_197"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q32.plan.txt index 0063a2a6a620..3fe019ccd451 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_20) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_20) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_20) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q33.plan.txt index c544e95a3541..fb7279d7b9b0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q33.plan.txt @@ -9,29 +9,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_manufact_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_25) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_25) - scan item + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_25) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_25"]) + partial aggregation over (i_manufact_id_25) + scan item partial aggregation over (i_manufact_id_103) final aggregation over (i_manufact_id_103) local exchange (GATHER, SINGLE, []) @@ -39,29 +36,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_manufact_id_103) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_130) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_130"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_130) - scan item + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_130) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_130"]) + partial aggregation over (i_manufact_id_130) + scan item partial aggregation over (i_manufact_id_210) final aggregation over (i_manufact_id_210) local exchange (GATHER, SINGLE, []) @@ -71,26 +65,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_181"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_197"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_237) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_237"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_237) - scan item + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_237) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_237"]) + partial aggregation over (i_manufact_id_237) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q34.plan.txt index 42dd07b77e7e..64d4ce451141 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q34.plan.txt @@ -8,18 +8,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ss_customer_sk, ss_ticket_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q35.plan.txt index 362715d1d4c4..d88f27b4c7b6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q35.plan.txt @@ -12,47 +12,41 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cs_ship_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q36.plan.txt index 5221892abb85..630dacce4c1e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q36.plan.txt @@ -8,17 +8,15 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q37.plan.txt index 008288ede797..3ae83aae01f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q38.plan.txt index 399554ef1411..94d514729231 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_7, c_last_name_8, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_55, c_last_name_56, d_date_18) final aggregation over (c_first_name_55, c_last_name_56, d_date_18) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_55, c_last_name_56, d_date_18) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["c_customer_sk_47"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim partial aggregation over (c_first_name_111, c_last_name_112, d_date_74) final aggregation over (c_first_name_111, c_last_name_112, d_date_74) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_111, c_last_name_112, d_date_74) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["c_customer_sk_103"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q39.plan.txt index a96d0b7e0a46..0761f10365a3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_11", "inv_warehouse_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_69, inv_item_sk_11, inv_warehouse_sk_12, w_warehouse_name_46) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_69", "inv_item_sk_11", "inv_warehouse_sk_12", "w_warehouse_name_46"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_69, inv_item_sk_11, inv_warehouse_sk_12, w_warehouse_name_46) + final aggregation over (d_moy_69, inv_item_sk_11, inv_warehouse_sk_12, w_warehouse_name_46) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_69", "inv_item_sk_11", "inv_warehouse_sk_12", "w_warehouse_name_46"]) + partial aggregation over (d_moy_69, inv_item_sk_11, inv_warehouse_sk_12, w_warehouse_name_46) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q40.plan.txt index 85412806878e..b7cf1dd0bae2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q42.plan.txt index d6a3c8cac82b..17cc12f10e9e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q44.plan.txt index bc31761eebff..bf870eae841a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) + partial aggregation over (ss_store_sk_6) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_86"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_29) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) + partial aggregation over (ss_item_sk_29) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_6) + final aggregation over (ss_store_sk_63) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_6) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_86"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_29) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_29) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_63) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_63"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_63) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_63"]) + partial aggregation over (ss_store_sk_63) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q45.plan.txt index 474216977dca..7dd52a5c0765 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q45.plan.txt @@ -8,22 +8,19 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q46.plan.txt index 6a7aabefa820..cd659271967f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q46.plan.txt @@ -2,35 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q47.plan.txt index ce0faaaab7ba..0911429ef31b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_17", "i_category_21", "s_company_name_109", "s_store_name_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_69, d_year_67, i_brand_17, i_category_21, s_company_name_109, s_store_name_97) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_69", "d_year_67", "i_brand_17", "i_category_21", "s_company_name_109", "s_store_name_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_69, d_year_67, i_brand_17, i_category_21, s_company_name_109, s_store_name_97) + final aggregation over (d_moy_69, d_year_67, i_brand_17, i_category_21, s_company_name_109, s_store_name_97) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_69", "d_year_67", "i_brand_17", "i_category_21", "s_company_name_109", "s_store_name_97"]) + partial aggregation over (d_moy_69, d_year_67, i_brand_17, i_category_21, s_company_name_109, s_store_name_97) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_141", "i_category_145", "s_company_name_233", "s_store_name_221"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_193, d_year_191, i_brand_141, i_category_145, s_company_name_233, s_store_name_221) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_193", "d_year_191", "i_brand_141", "i_category_145", "s_company_name_233", "s_store_name_221"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_193, d_year_191, i_brand_141, i_category_145, s_company_name_233, s_store_name_221) + final aggregation over (d_moy_193, d_year_191, i_brand_141, i_category_145, s_company_name_233, s_store_name_221) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_193", "d_year_191", "i_brand_141", "i_category_145", "s_company_name_233", "s_store_name_221"]) + partial aggregation over (d_moy_193, d_year_191, i_brand_141, i_category_145, s_company_name_233, s_store_name_221) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q48.plan.txt index 3f5a3466a293..d002184141c7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q48.plan.txt @@ -5,20 +5,18 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q49.plan.txt index eb091b77b5f5..6b7ece910520 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q49.plan.txt @@ -3,59 +3,53 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_20", "expr_24", "rank", "rank_22", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_20, expr_24, rank, rank_22, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_item_sk) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_20, expr_24, rank, rank_22, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_74", "expr_78", "rank_76", "rank_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_74, expr_78, rank_76, rank_77) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_74, expr_78, rank_76, rank_77) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_128", "expr_133", "rank_130", "rank_131", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_128, expr_133, rank_130, rank_131, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_item_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_128, expr_133, rank_130, rank_131, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q50.plan.txt index 89f63e55a780..9f5732a36de7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q50.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_city", "s_company_id", "s_county", "s_state", "s_store_name", "s_street_name", "s_street_number", "s_street_type", "s_suite_number", "s_zip"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) + partial aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q51.plan.txt index 0740bb6b16e6..572cebe3eb35 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_10, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_10", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_10, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_10, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_10", "ss_item_sk"]) + partial aggregation over (d_date_10, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q52.plan.txt index ab4fb497ba27..33752e693e6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q53.plan.txt index 8fec7b4a24ec..8b8179689d06 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q53.plan.txt @@ -2,24 +2,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - partial aggregation over (d_qoy, i_manufact_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_store_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_sk"]) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q54.plan.txt index 2572b1c3d263..a7911d69a280 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_116) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_116"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_116) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_116) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_116"]) + partial aggregation over (expr_116) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q55.plan.txt index ad84f78b05f8..0322599f01e5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q56.plan.txt index d1026992d606..0f2f93f9a976 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q56.plan.txt @@ -9,29 +9,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_13) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_13"]) + partial aggregation over (i_item_id_13) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_91) final aggregation over (i_item_id_91) local exchange (GATHER, SINGLE, []) @@ -39,29 +36,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_91) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_118) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_118) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_118) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_118"]) + partial aggregation over (i_item_id_118) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_198) final aggregation over (i_item_id_198) local exchange (GATHER, SINGLE, []) @@ -69,26 +63,23 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_198) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_225) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_225"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_225) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_225) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_225"]) + partial aggregation over (i_item_id_225) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_181"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q57.plan.txt index 7cab3797a1b1..4bed99d3b879 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_109", "i_brand_17", "i_category_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_109, d_moy_80, d_year_78, i_brand_17, i_category_21) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_109", "d_moy_80", "d_year_78", "i_brand_17", "i_category_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_109, d_moy_80, d_year_78, i_brand_17, i_category_21) + final aggregation over (cc_name_109, d_moy_80, d_year_78, i_brand_17, i_category_21) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_109", "d_moy_80", "d_year_78", "i_brand_17", "i_category_21"]) + partial aggregation over (cc_name_109, d_moy_80, d_year_78, i_brand_17, i_category_21) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_245", "i_brand_153", "i_category_157"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_245, d_moy_216, d_year_214, i_brand_153, i_category_157) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_245", "d_moy_216", "d_year_214", "i_brand_153", "i_category_157"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_245, d_moy_216, d_year_214, i_brand_153, i_category_157) + final aggregation over (cc_name_245, d_moy_216, d_year_214, i_brand_153, i_category_157) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_245", "d_moy_216", "d_year_214", "i_brand_153", "i_category_157"]) + partial aggregation over (cc_name_245, d_moy_216, d_year_214, i_brand_153, i_category_157) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q58.plan.txt index 8a6e79c33754..fb65ed72a690 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q58.plan.txt @@ -8,81 +8,46 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_75) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_134) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_134"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_134) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_134) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_134"]) + partial aggregation over (d_date_134) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_203) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_203"]) partial aggregation over (i_item_id_203) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_262) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_262"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_262) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_202"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_8) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_8) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_262) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_262"]) + partial aggregation over (d_date_262) join (INNER, REPLICATED, can skip output duplicates): scan date_dim local exchange (GATHER, SINGLE, []) @@ -91,5 +56,32 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_202"]) scan item + final aggregation over (i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id"]) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_8) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_8"]) + partial aggregation over (d_date_8) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q59.plan.txt index e919b701b5bb..719af5ccc77b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q59.plan.txt @@ -2,53 +2,47 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["d_week_seq", "s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_211", "s_store_id_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_80, ss_store_sk_55) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_80", "ss_store_sk_55"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_80, ss_store_sk_55) - final aggregation over (d_day_name_90, d_week_seq_80, ss_store_sk_55) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_90", "d_week_seq_80", "ss_store_sk_55"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_90, d_week_seq_80, ss_store_sk_55) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_80, ss_store_sk_55) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_80", "ss_store_sk_55"]) + partial aggregation over (d_week_seq_80, ss_store_sk_55) + final aggregation over (d_day_name_90, d_week_seq_80, ss_store_sk_55) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_90", "d_week_seq_80", "ss_store_sk_55"]) + partial aggregation over (d_day_name_90, d_week_seq_80, ss_store_sk_55) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q60.plan.txt index 7b0d31a2a635..c573da2e1725 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q60.plan.txt @@ -9,29 +9,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_13) - scan item + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_13"]) + partial aggregation over (i_item_id_13) + scan item partial aggregation over (i_item_id_91) final aggregation over (i_item_id_91) local exchange (GATHER, SINGLE, []) @@ -39,29 +36,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_91) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_118) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_118) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_118) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_118"]) + partial aggregation over (i_item_id_118) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_198) final aggregation over (i_item_id_198) local exchange (GATHER, SINGLE, []) @@ -69,26 +63,23 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_198) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_225) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_225"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_225) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_225) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_225"]) + partial aggregation over (i_item_id_225) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_181"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q61.plan.txt index 28da7501da8e..958cb1b32e1b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q61.plan.txt @@ -5,57 +5,53 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_129"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk_112"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q62.plan.txt index a00a28581512..61172c74fc4f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q63.plan.txt index eeebbe56a704..23a54b451e8f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q63.plan.txt @@ -2,24 +2,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - partial aggregation over (d_moy, i_manager_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_store_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_sk"]) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q64.plan.txt index 2907b89404be..0ab88e33c808 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q64.plan.txt @@ -3,195 +3,177 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_122, ca_street_name, ca_street_name_119, ca_street_number, ca_street_number_118, ca_zip, ca_zip_125, d_year, d_year_22, d_year_53, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_122", "ca_street_name", "ca_street_name_119", "ca_street_number", "ca_street_number_118", "ca_zip", "ca_zip_125", "d_year", "d_year_22", "d_year_53", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city, ca_city_122, ca_street_name, ca_street_name_119, ca_street_number, ca_street_number_118, ca_zip, ca_zip_125, d_year, d_year_22, d_year_53, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_87"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + final aggregation over (ca_city, ca_city_122, ca_street_name, ca_street_name_119, ca_street_number, ca_street_number_118, ca_zip, ca_zip_125, d_year, d_year_22, d_year_53, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_122", "ca_street_name", "ca_street_name_119", "ca_street_number", "ca_street_number_118", "ca_zip", "ca_zip_125", "d_year", "d_year_22", "d_year_53", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_122, ca_street_name, ca_street_name_119, ca_street_number, ca_street_number_118, ca_zip, ca_zip_125, d_year, d_year_22, d_year_53, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_87"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_name_375", "s_zip_395", "ss_item_sk_154"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_491, ca_city_507, ca_street_name_488, ca_street_name_504, ca_street_number_487, ca_street_number_503, ca_zip_494, ca_zip_510, d_year_283, d_year_314, d_year_345, i_product_name_550, s_store_name_375, s_zip_395, ss_item_sk_154) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_491", "ca_city_507", "ca_street_name_488", "ca_street_name_504", "ca_street_number_487", "ca_street_number_503", "ca_zip_494", "ca_zip_510", "d_year_283", "d_year_314", "d_year_345", "i_product_name_550", "s_store_name_375", "s_zip_395", "ss_item_sk_154"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city_491, ca_city_507, ca_street_name_488, ca_street_name_504, ca_street_number_487, ca_street_number_503, ca_zip_494, ca_zip_510, d_year_283, d_year_314, d_year_345, i_product_name_550, s_store_name_375, s_zip_395, ss_item_sk_154) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_435"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_404"]) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_name_375", "s_zip_395", "ss_item_sk_154"]) + final aggregation over (ca_city_491, ca_city_507, ca_street_name_488, ca_street_name_504, ca_street_number_487, ca_street_number_503, ca_zip_494, ca_zip_510, d_year_283, d_year_314, d_year_345, i_product_name_550, s_store_name_375, s_zip_395, ss_item_sk_154) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_491", "ca_city_507", "ca_street_name_488", "ca_street_name_504", "ca_street_number_487", "ca_street_number_503", "ca_zip_494", "ca_zip_510", "d_year_283", "d_year_314", "d_year_345", "i_product_name_550", "s_store_name_375", "s_zip_395", "ss_item_sk_154"]) + partial aggregation over (ca_city_491, ca_city_507, ca_street_name_488, ca_street_name_504, ca_street_number_487, ca_street_number_503, ca_zip_494, ca_zip_510, d_year_283, d_year_314, d_year_345, i_product_name_550, s_store_name_375, s_zip_395, ss_item_sk_154) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_435"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_404"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_402"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk_155"]) - join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_402"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk_155"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_156"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_156"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_154", "ss_ticket_number_161"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_item_sk_154", "ss_ticket_number_161"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_181", "sr_ticket_number_188"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_218) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_218"]) - partial aggregation over (cs_item_sk_218) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_218", "cs_order_number_220"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_243", "cr_order_number_257"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_423"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["sr_item_sk_181", "sr_ticket_number_188"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk_218) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_218"]) + partial aggregation over (cs_item_sk_218) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_218", "cs_order_number_220"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_243", "cr_order_number_257"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_423"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q65.plan.txt index eaf127c7f5a0..744c84a2143a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_10, ss_store_sk_15) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_10", "ss_store_sk_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_10, ss_store_sk_15) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_10, ss_store_sk_15) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q66.plan.txt index 2abe02a49224..785f1198b31c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_381, d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) final aggregation over (d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_91", "w_city_76", "w_country_80", "w_county_77", "w_state_78", "w_warehouse_name_70", "w_warehouse_sq_ft_71"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) - final aggregation over (d_moy_93, d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_93", "d_year_91", "w_city_76", "w_country_80", "w_county_77", "w_state_78", "w_warehouse_name_70", "w_warehouse_sq_ft_71"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_93, d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) + partial aggregation over (d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) + final aggregation over (d_moy_93, d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_93", "d_year_91", "w_city_76", "w_country_80", "w_county_77", "w_state_78", "w_warehouse_name_70", "w_warehouse_sq_ft_71"]) + partial aggregation over (d_moy_93, d_year_91, w_city_76, w_country_80, w_county_77, w_state_78, w_warehouse_name_70, w_warehouse_sq_ft_71) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q68.plan.txt index 0f1bf39938ff..6c71f017bae6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q68.plan.txt @@ -2,34 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk", "ca_city", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk", "ca_city", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q69.plan.txt index 2fdcca1cc11c..18406802cc1b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q69.plan.txt @@ -9,49 +9,43 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_ship_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q70.plan.txt index d6d865362e78..7f94b5469eb9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_98) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_57) final aggregation over (s_state_57) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_57"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_57) + partial aggregation over (s_state_57) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q71.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q71.plan.txt index c5dc8d44d272..02c955d77dfd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q71.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q71.plan.txt @@ -29,5 +29,4 @@ remote exchange (GATHER, SINGLE, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["t_time_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan time_dim + scan time_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q72.plan.txt index 17448606409b..8c52b501aa32 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q72.plan.txt @@ -3,50 +3,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, i_item_desc, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "i_item_desc", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, i_item_desc, w_warehouse_name) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - join (LEFT, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (d_week_seq, i_item_desc, w_warehouse_name) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + join (LEFT, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["inv_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q73.plan.txt index ab6eee0fb93b..8ca487c16558 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q73.plan.txt @@ -8,18 +8,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ss_customer_sk, ss_ticket_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q74.plan.txt index 294b48c00816..13345b1ee1f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q74.plan.txt @@ -2,79 +2,67 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_75"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_75, c_first_name_82, c_last_name_83, d_year_128) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_75", "c_first_name_82", "c_last_name_83", "d_year_128"]) - partial aggregation over (c_customer_id_75, c_first_name_82, c_last_name_83, d_year_128) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (c_customer_id_75, c_first_name_82, c_last_name_83, d_year_128) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_75", "c_first_name_82", "c_last_name_83", "d_year_128"]) + partial aggregation over (c_customer_id_75, c_first_name_82, c_last_name_83, d_year_128) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_97"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_74"]) + scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id"]) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) join (INNER, REPLICATED): scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_534"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_534, c_first_name_541, c_last_name_542, d_year_598) + final aggregation over (c_customer_id_534, c_first_name_541, c_last_name_542, d_year_598) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_534", "c_first_name_541", "c_last_name_542", "d_year_598"]) + partial aggregation over (c_customer_id_534, c_first_name_541, c_last_name_542, d_year_598) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk_533"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_557"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_347"]) + final aggregation over (c_customer_id_347, c_first_name_354, c_last_name_355, d_year_411) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_534", "c_first_name_541", "c_last_name_542", "d_year_598"]) - partial aggregation over (c_customer_id_534, c_first_name_541, c_last_name_542, d_year_598) + remote exchange (REPARTITION, HASH, ["c_customer_id_347", "c_first_name_354", "c_last_name_355", "d_year_411"]) + partial aggregation over (c_customer_id_347, c_first_name_354, c_last_name_355, d_year_411) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_533"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_346"]) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_557"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_347"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_347, c_first_name_354, c_last_name_355, d_year_411) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_347", "c_first_name_354", "c_last_name_355", "d_year_411"]) - partial aggregation over (c_customer_id_347, c_first_name_354, c_last_name_355, d_year_411) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk_346"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_370"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_370"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q75.plan.txt index e89787932b7a..20311d91862a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_14, expr, expr_22, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_14, expr, expr_22, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_34", "i_category_id_38", "i_class_id_36", "i_manufact_id_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_58, expr_87, expr_88, i_brand_id_34, i_category_id_38, i_class_id_36, i_manufact_id_40) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_58, expr_87, expr_88, i_brand_id_34, i_category_id_38, i_class_id_36, i_manufact_id_40) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_100", "i_category_id_104", "i_class_id_102", "i_manufact_id_106"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_124, expr_153, expr_154, i_brand_id_100, i_category_id_104, i_class_id_102, i_manufact_id_106) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_124, expr_153, expr_154, i_brand_id_100, i_category_id_104, i_class_id_102, i_manufact_id_106) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item single aggregation over (d_year_157, i_brand_id_158, i_category_id_160, i_class_id_159, i_manufact_id_161) final aggregation over (d_year_157, i_brand_id_158, i_category_id_160, i_class_id_159, i_manufact_id_161, sales_amt_163, sales_cnt_162) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_216", "i_category_id_220", "i_class_id_218", "i_manufact_id_222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_240, expr_296, expr_297, i_brand_id_216, i_category_id_220, i_class_id_218, i_manufact_id_222) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_266", "cr_order_number_280"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_185", "cs_order_number_187"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_240, expr_296, expr_297, i_brand_id_216, i_category_id_220, i_class_id_218, i_manufact_id_222) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_266", "cr_order_number_280"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_185", "cs_order_number_187"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_332", "i_category_id_336", "i_class_id_334", "i_manufact_id_338"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_356, expr_405, expr_406, i_brand_id_332, i_category_id_336, i_class_id_334, i_manufact_id_338) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_382", "sr_ticket_number_389"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_299", "ss_ticket_number_306"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_356, expr_405, expr_406, i_brand_id_332, i_category_id_336, i_class_id_334, i_manufact_id_338) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_382", "sr_ticket_number_389"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_299", "ss_ticket_number_306"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["i_brand_id_452", "i_category_id_456", "i_class_id_454", "i_manufact_id_458"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_476, expr_529, expr_530, i_brand_id_452, i_category_id_456, i_class_id_454, i_manufact_id_458) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_502", "wr_order_number_513"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_409", "ws_order_number_423"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (d_year_476, expr_529, expr_530, i_brand_id_452, i_category_id_456, i_class_id_454, i_manufact_id_458) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_502", "wr_order_number_513"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_409", "ws_order_number_423"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q76.plan.txt index 393fbfd02336..d7b6d9db21ac 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q76.plan.txt @@ -3,40 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_11", "d_year_10", "expr_144", "expr_145", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_11, d_year_10, expr_144, expr_145, i_category_6) + partial aggregation over (d_qoy_11, d_year_10, expr_144, expr_145, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["d_qoy_56", "d_year_52", "expr_150", "expr_151", "i_category_33"]) partial aggregation over (d_qoy_56, d_year_52, expr_150, expr_151, i_category_33) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - remote exchange (REPARTITION, HASH, ["d_qoy_120", "d_year_116", "expr_141", "expr_143", "i_category_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_120, d_year_116, expr_141, expr_143, i_category_97) - join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_21"]) + scan item + remote exchange (REPARTITION, HASH, ["d_qoy_120", "d_year_116", "expr_141", "expr_143", "i_category_97"]) + partial aggregation over (d_qoy_120, d_year_116, expr_141, expr_143, i_category_97) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q78.plan.txt index 190bea4850cf..c21aa577ba08 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q78.plan.txt @@ -8,12 +8,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -27,12 +25,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +39,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q81.plan.txt index cc48bf7e28db..ce0e336b201c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q81.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) - partial aggregation over (ca_state, cr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (ca_state, cr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) + partial aggregation over (ca_state, cr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_95) - final aggregation over (ca_state_95, cr_returning_customer_sk_31) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_95", "cr_returning_customer_sk_31"]) - partial aggregation over (ca_state_95, cr_returning_customer_sk_31) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_87"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ca_state_95) + final aggregation over (ca_state_95, cr_returning_customer_sk_31) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_95", "cr_returning_customer_sk_31"]) + partial aggregation over (ca_state_95, cr_returning_customer_sk_31) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_87"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_34"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q82.plan.txt index 358e01cc84f0..ab193ba29af9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q83.plan.txt index 2e6d37fdd4e8..87e080b8e1e2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q83.plan.txt @@ -8,99 +8,87 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_80) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_139) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_139"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_139) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_174) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_174"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_174) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_139) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_139"]) + partial aggregation over (d_date_139) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_174) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_174"]) + partial aggregation over (d_week_seq_174) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_213) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_213"]) partial aggregation over (i_item_id_213) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_212"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_272) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_272"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_272) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_307) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_307"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_307) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_272) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_272"]) + partial aggregation over (d_date_272) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_307) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_307"]) + partial aggregation over (d_week_seq_307) + scan date_dim final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_8) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_8) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_43) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_43"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_43) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_8) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_8"]) + partial aggregation over (d_date_8) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_43) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_43"]) + partial aggregation over (d_week_seq_43) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q85.plan.txt index 09809dcef670..2ae4cae3b89b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q85.plan.txt @@ -3,41 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q86.plan.txt index f880adf8bf2d..990089879464 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q87.plan.txt index 399554ef1411..94d514729231 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_7, c_last_name_8, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_55, c_last_name_56, d_date_18) final aggregation over (c_first_name_55, c_last_name_56, d_date_18) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_55, c_last_name_56, d_date_18) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["c_customer_sk_47"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim partial aggregation over (c_first_name_111, c_last_name_112, d_date_74) final aggregation over (c_first_name_111, c_last_name_112, d_date_74) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_111, c_last_name_112, d_date_74) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["c_customer_sk_103"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q89.plan.txt index 4446fe5ca255..2737c0093fbb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q91.plan.txt index 65d6cda93e78..c113b8bda923 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q91.plan.txt @@ -11,24 +11,21 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q92.plan.txt index 7c55cbd98e00..4d76e724d197 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_8) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_8) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q94.plan.txt index 66f3ade77f16..259bf30db48e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q94.plan.txt @@ -8,27 +8,24 @@ final aggregation over () partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_25, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_site + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q95.plan.txt index 6b72edeb5d9f..ea818962db9f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q95.plan.txt @@ -4,48 +4,42 @@ final aggregation over () partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ws_order_number_25) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ws_order_number_25) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["ws_order_number_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_order_number_63"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - final aggregation over (ws_order_number_109) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ws_order_number_109) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ws_order_number_25) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ws_order_number_25) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["ws_order_number_25"]) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_order_number_63"]) + scan web_sales + final aggregation over (ws_order_number_109) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ws_order_number_109) + join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["ws_order_number_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["ws_order_number_109"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_order_number_147"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + remote exchange (REPARTITION, HASH, ["wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_order_number_147"]) + scan web_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q98.plan.txt index aa5503f47d8e..a6a04beb1e8b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q98.plan.txt @@ -3,20 +3,17 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q99.plan.txt index e17b712a1894..6e2c1451c321 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q01.plan.txt index 8dc6946909e2..da4afcb7be2b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q01.plan.txt @@ -4,42 +4,37 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (sr_store_sk_16) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_16) - final aggregation over (sr_customer_sk_12, sr_store_sk_16) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_12", "sr_store_sk_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_12, sr_store_sk_16) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_16) + final aggregation over (sr_customer_sk_12, sr_store_sk_16) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_12", "sr_store_sk_16"]) + partial aggregation over (sr_customer_sk_12, sr_store_sk_16) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q02.plan.txt index db79dd1da69e..ded78380a2fc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_226"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_129) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_129"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_129) - final aggregation over (d_day_name_139, d_week_seq_129) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_139", "d_week_seq_129"]) - partial aggregation over (d_day_name_139, d_week_seq_129) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_129) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_175"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_129"]) + partial aggregation over (d_week_seq_129) + final aggregation over (d_day_name_139, d_week_seq_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_139", "d_week_seq_129"]) + partial aggregation over (d_day_name_139, d_week_seq_129) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_175"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q04.plan.txt index 2b1977a14f13..420fa6a923ec 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q04.plan.txt @@ -4,117 +4,99 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_877"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_890, c_customer_id_877, c_email_address_892, c_first_name_884, c_last_name_885, c_login_891, c_preferred_cust_flag_886, d_year_940) + final aggregation over (c_birth_country_890, c_customer_id_877, c_email_address_892, c_first_name_884, c_last_name_885, c_login_891, c_preferred_cust_flag_886, d_year_940) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_890", "c_customer_id_877", "c_email_address_892", "c_first_name_884", "c_last_name_885", "c_login_891", "c_preferred_cust_flag_886", "d_year_940"]) + partial aggregation over (c_birth_country_890, c_customer_id_877, c_email_address_892, c_first_name_884, c_last_name_885, c_login_891, c_preferred_cust_flag_886, d_year_940) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_900"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_876"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1595"]) + final aggregation over (c_birth_country_1608, c_customer_id_1595, c_email_address_1610, c_first_name_1602, c_last_name_1603, c_login_1609, c_preferred_cust_flag_1604, d_year_1658) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_890", "c_customer_id_877", "c_email_address_892", "c_first_name_884", "c_last_name_885", "c_login_891", "c_preferred_cust_flag_886", "d_year_940"]) - partial aggregation over (c_birth_country_890, c_customer_id_877, c_email_address_892, c_first_name_884, c_last_name_885, c_login_891, c_preferred_cust_flag_886, d_year_940) + remote exchange (REPARTITION, HASH, ["c_birth_country_1608", "c_customer_id_1595", "c_email_address_1610", "c_first_name_1602", "c_last_name_1603", "c_login_1609", "c_preferred_cust_flag_1604", "d_year_1658"]) + partial aggregation over (c_birth_country_1608, c_customer_id_1595, c_email_address_1610, c_first_name_1602, c_last_name_1603, c_login_1609, c_preferred_cust_flag_1604, d_year_1658) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_900"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1619"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_876"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1595"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1608, c_customer_id_1595, c_email_address_1610, c_first_name_1602, c_last_name_1603, c_login_1609, c_preferred_cust_flag_1604, d_year_1658) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1608", "c_customer_id_1595", "c_email_address_1610", "c_first_name_1602", "c_last_name_1603", "c_login_1609", "c_preferred_cust_flag_1604", "d_year_1658"]) - partial aggregation over (c_birth_country_1608, c_customer_id_1595, c_email_address_1610, c_first_name_1602, c_last_name_1603, c_login_1609, c_preferred_cust_flag_1604, d_year_1658) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1619"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1594"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1594"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_568"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_581, c_customer_id_568, c_email_address_583, c_first_name_575, c_last_name_576, c_login_582, c_preferred_cust_flag_577, d_year_631) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_581", "c_customer_id_568", "c_email_address_583", "c_first_name_575", "c_last_name_576", "c_login_582", "c_preferred_cust_flag_577", "d_year_631"]) - partial aggregation over (c_birth_country_581, c_customer_id_568, c_email_address_583, c_first_name_575, c_last_name_576, c_login_582, c_preferred_cust_flag_577, d_year_631) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_591"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_567"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1286"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1299, c_customer_id_1286, c_email_address_1301, c_first_name_1293, c_last_name_1294, c_login_1300, c_preferred_cust_flag_1295, d_year_1349) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1299", "c_customer_id_1286", "c_email_address_1301", "c_first_name_1293", "c_last_name_1294", "c_login_1300", "c_preferred_cust_flag_1295", "d_year_1349"]) - partial aggregation over (c_birth_country_1299, c_customer_id_1286, c_email_address_1301, c_first_name_1293, c_last_name_1294, c_login_1300, c_preferred_cust_flag_1295, d_year_1349) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1310"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1285"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_170"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_183, c_customer_id_170, c_email_address_185, c_first_name_177, c_last_name_178, c_login_184, c_preferred_cust_flag_179, d_year_222) + final aggregation over (c_birth_country_581, c_customer_id_568, c_email_address_583, c_first_name_575, c_last_name_576, c_login_582, c_preferred_cust_flag_577, d_year_631) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_183", "c_customer_id_170", "c_email_address_185", "c_first_name_177", "c_last_name_178", "c_login_184", "c_preferred_cust_flag_179", "d_year_222"]) - partial aggregation over (c_birth_country_183, c_customer_id_170, c_email_address_185, c_first_name_177, c_last_name_178, c_login_184, c_preferred_cust_flag_179, d_year_222) + remote exchange (REPARTITION, HASH, ["c_birth_country_581", "c_customer_id_568", "c_email_address_583", "c_first_name_575", "c_last_name_576", "c_login_582", "c_preferred_cust_flag_577", "d_year_631"]) + partial aggregation over (c_birth_country_581, c_customer_id_568, c_email_address_583, c_first_name_575, c_last_name_576, c_login_582, c_preferred_cust_flag_577, d_year_631) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_193"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_591"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_567"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1286"]) + final aggregation over (c_birth_country_1299, c_customer_id_1286, c_email_address_1301, c_first_name_1293, c_last_name_1294, c_login_1300, c_preferred_cust_flag_1295, d_year_1349) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1299", "c_customer_id_1286", "c_email_address_1301", "c_first_name_1293", "c_last_name_1294", "c_login_1300", "c_preferred_cust_flag_1295", "d_year_1349"]) + partial aggregation over (c_birth_country_1299, c_customer_id_1286, c_email_address_1301, c_first_name_1293, c_last_name_1294, c_login_1300, c_preferred_cust_flag_1295, d_year_1349) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1310"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_169"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_1285"]) scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_170"]) + final aggregation over (c_birth_country_183, c_customer_id_170, c_email_address_185, c_first_name_177, c_last_name_178, c_login_184, c_preferred_cust_flag_179, d_year_222) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_183", "c_customer_id_170", "c_email_address_185", "c_first_name_177", "c_last_name_178", "c_login_184", "c_preferred_cust_flag_179", "d_year_222"]) + partial aggregation over (c_birth_country_183, c_customer_id_170, c_email_address_185, c_first_name_177, c_last_name_178, c_login_184, c_preferred_cust_flag_179, d_year_222) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_193"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_169"]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q05.plan.txt index 9ab2adedbf44..4155bb638c70 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_92", "ws_order_number_106"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q06.plan.txt index 52553713d3ad..2a72afdda3fa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q06.plan.txt @@ -9,33 +9,29 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_15) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_15) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_15) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_15"]) + partial aggregation over (d_month_seq_15) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -44,9 +40,8 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_55) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_55"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_55) - scan item + partial aggregation over (i_category_55) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q07.plan.txt index d209bd09ae61..32f4763b443d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q07.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q08.plan.txt index b833e024e439..432e37f5b19c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_39"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_40"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_31) - scan customer_address + partial aggregation over (ca_zip_31) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_34) - final aggregation over (ca_zip_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_19"]) - partial aggregation over (ca_zip_19) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_34) + final aggregation over (ca_zip_19) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_19"]) + partial aggregation over (ca_zip_19) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_10"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q10.plan.txt index a53dd7834678..7d60d2a5fdd5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q10.plan.txt @@ -3,54 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q11.plan.txt index b71d7c228bdd..3edd5dbb25a4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q11.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_97, c_customer_id_84, c_email_address_99, c_first_name_91, c_last_name_92, c_login_98, c_preferred_cust_flag_93, d_year_136) + final aggregation over (c_birth_country_97, c_customer_id_84, c_email_address_99, c_first_name_91, c_last_name_92, c_login_98, c_preferred_cust_flag_93, d_year_136) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_97", "c_customer_id_84", "c_email_address_99", "c_first_name_91", "c_last_name_92", "c_login_98", "c_preferred_cust_flag_93", "d_year_136"]) + partial aggregation over (c_birth_country_97, c_customer_id_84, c_email_address_99, c_first_name_91, c_last_name_92, c_login_98, c_preferred_cust_flag_93, d_year_136) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_107"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_83"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_372"]) + final aggregation over (c_birth_country_385, c_customer_id_372, c_email_address_387, c_first_name_379, c_last_name_380, c_login_386, c_preferred_cust_flag_381, d_year_435) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_97", "c_customer_id_84", "c_email_address_99", "c_first_name_91", "c_last_name_92", "c_login_98", "c_preferred_cust_flag_93", "d_year_136"]) - partial aggregation over (c_birth_country_97, c_customer_id_84, c_email_address_99, c_first_name_91, c_last_name_92, c_login_98, c_preferred_cust_flag_93, d_year_136) + remote exchange (REPARTITION, HASH, ["c_birth_country_385", "c_customer_id_372", "c_email_address_387", "c_first_name_379", "c_last_name_380", "c_login_386", "c_preferred_cust_flag_381", "d_year_435"]) + partial aggregation over (c_birth_country_385, c_customer_id_372, c_email_address_387, c_first_name_379, c_last_name_380, c_login_386, c_preferred_cust_flag_381, d_year_435) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_107"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_396"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_372"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_385, c_customer_id_372, c_email_address_387, c_first_name_379, c_last_name_380, c_login_386, c_preferred_cust_flag_381, d_year_435) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_385", "c_customer_id_372", "c_email_address_387", "c_first_name_379", "c_last_name_380", "c_login_386", "c_preferred_cust_flag_381", "d_year_435"]) - partial aggregation over (c_birth_country_385, c_customer_id_372, c_email_address_387, c_first_name_379, c_last_name_380, c_login_386, c_preferred_cust_flag_381, d_year_435) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_396"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_371"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_371"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_571"]) + final aggregation over (c_birth_country_584, c_customer_id_571, c_email_address_586, c_first_name_578, c_last_name_579, c_login_585, c_preferred_cust_flag_580, d_year_634) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + remote exchange (REPARTITION, HASH, ["c_birth_country_584", "c_customer_id_571", "c_email_address_586", "c_first_name_578", "c_last_name_579", "c_login_585", "c_preferred_cust_flag_580", "d_year_634"]) + partial aggregation over (c_birth_country_584, c_customer_id_571, c_email_address_586, c_first_name_578, c_last_name_579, c_login_585, c_preferred_cust_flag_580, d_year_634) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_595"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_571"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_584, c_customer_id_571, c_email_address_586, c_first_name_578, c_last_name_579, c_login_585, c_preferred_cust_flag_580, d_year_634) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_584", "c_customer_id_571", "c_email_address_586", "c_first_name_578", "c_last_name_579", "c_login_585", "c_preferred_cust_flag_580", "d_year_634"]) - partial aggregation over (c_birth_country_584, c_customer_id_571, c_email_address_586, c_first_name_578, c_last_name_579, c_login_585, c_preferred_cust_flag_580, d_year_634) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_595"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_570"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_570"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q12.plan.txt index 14d23e717ece..6f94c98c5224 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q12.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q14.plan.txt index 5e73c397329e..fe1028ad939b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_15) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_15"]) - partial aggregation over (i_item_sk_15) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_22", "i_category_id_26", "i_class_id_24"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_76", "i_category_id_80", "i_class_id_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_76, i_category_id_80, i_class_id_78) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_135", "i_category_id_139", "i_class_id_137"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_135, i_category_id_139, i_class_id_137) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_194", "i_category_id_198", "i_class_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_194, i_category_id_198, i_class_id_196) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_15) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_15"]) + partial aggregation over (i_item_sk_15) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_22", "i_category_id_26", "i_class_id_24"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_76", "i_category_id_80", "i_class_id_78"]) + partial aggregation over (i_brand_id_76, i_category_id_80, i_class_id_78) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_135", "i_category_id_139", "i_class_id_137"]) + partial aggregation over (i_brand_id_135, i_category_id_139, i_class_id_137) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_194", "i_category_id_198", "i_class_id_196"]) + partial aggregation over (i_brand_id_194, i_category_id_198, i_class_id_196) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_488, i_category_id_492, i_class_id_490) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_488", "i_category_id_492", "i_class_id_490"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_488, i_category_id_492, i_class_id_490) + partial aggregation over (i_brand_id_488, i_category_id_492, i_class_id_490) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_481"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_539) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_539"]) - partial aggregation over (i_item_sk_539) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_546", "i_category_id_550", "i_class_id_548"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_564, category_id_566, class_id_565) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_603", "i_category_id_607", "i_class_id_605"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_603, i_category_id_607, i_class_id_605) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_696", "i_category_id_700", "i_class_id_698"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_696, i_category_id_700, i_class_id_698) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_789", "i_category_id_793", "i_class_id_791"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_789, i_category_id_793, i_class_id_791) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_481"]) + scan item + final aggregation over (i_item_sk_539) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_539"]) + partial aggregation over (i_item_sk_539) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_546", "i_category_id_550", "i_class_id_548"]) + scan item + final aggregation over (brand_id_564, category_id_566, class_id_565) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_603", "i_category_id_607", "i_class_id_605"]) + partial aggregation over (i_brand_id_603, i_category_id_607, i_class_id_605) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_696", "i_category_id_700", "i_class_id_698"]) + partial aggregation over (i_brand_id_696, i_category_id_700, i_class_id_698) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_789", "i_category_id_793", "i_class_id_791"]) + partial aggregation over (i_brand_id_789, i_category_id_793, i_class_id_791) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -183,66 +171,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1090, i_category_id_1094, i_class_id_1092) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1090", "i_category_id_1094", "i_class_id_1092"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1090, i_category_id_1094, i_class_id_1092) + partial aggregation over (i_brand_id_1090, i_category_id_1094, i_class_id_1092) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_1083"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_1141) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_1141"]) - partial aggregation over (i_item_sk_1141) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_1148", "i_category_id_1152", "i_class_id_1150"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_1166, category_id_1168, class_id_1167) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_1205", "i_category_id_1209", "i_class_id_1207"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1205, i_category_id_1209, i_class_id_1207) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1298", "i_category_id_1302", "i_class_id_1300"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1298, i_category_id_1302, i_class_id_1300) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1391", "i_category_id_1395", "i_class_id_1393"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1391, i_category_id_1395, i_class_id_1393) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_1083"]) + scan item + final aggregation over (i_item_sk_1141) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_1141"]) + partial aggregation over (i_item_sk_1141) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_1148", "i_category_id_1152", "i_class_id_1150"]) + scan item + final aggregation over (brand_id_1166, category_id_1168, class_id_1167) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_1205", "i_category_id_1209", "i_class_id_1207"]) + partial aggregation over (i_brand_id_1205, i_category_id_1209, i_class_id_1207) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1298", "i_category_id_1302", "i_class_id_1300"]) + partial aggregation over (i_brand_id_1298, i_category_id_1302, i_class_id_1300) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1391", "i_category_id_1395", "i_class_id_1393"]) + partial aggregation over (i_brand_id_1391, i_category_id_1395, i_class_id_1393) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q15.plan.txt index 4a9d57b8e143..f359fbefb40f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q16.plan.txt index 514a586ee0de..ac81595cda39 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q16.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (cr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_26, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_26", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_26, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_26, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_26", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_26, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q17.plan.txt index 326846b8f36d..47c73f484403 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q17.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_state) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q18.plan.txt index 4e39a11a0e9a..f338fb880363 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q18.plan.txt @@ -20,16 +20,13 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk_3"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q19.plan.txt index c3cd8014d9b4..5230f46699ba 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q19.plan.txt @@ -6,27 +6,24 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q20.plan.txt index 2dce6eadbb48..852a9d2a3e5c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q20.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q21.plan.txt index e8067ff5be06..50e40917c430 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q23.plan.txt index ae9b7d2ec96f..56d143391099 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_8, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_8", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_8, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_8, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_8", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_8, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_47) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_47) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_47) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_47"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_77) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_196) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_196) - final aggregation over (d_date_222, ss_item_sk_196, substr$gid_279) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_222", "ss_item_sk_196", "substr$gid_279"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_222, ss_item_sk_196, substr$gid_279) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_196) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_196"]) + partial aggregation over (ss_item_sk_196) + final aggregation over (d_date_222, ss_item_sk_196, substr$gid_279) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_222", "ss_item_sk_196", "substr$gid_279"]) + partial aggregation over (d_date_222, ss_item_sk_196, substr$gid_279) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_288) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_288) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_288) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_288"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_311"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_338) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_338"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_361"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q24.plan.txt index 3147f15c942c..5a1ffc003e35 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q24.plan.txt @@ -5,39 +5,34 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -55,24 +50,20 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk_19"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_18", "ss_ticket_number_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_44", "sr_ticket_number_51"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_122"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_152", "upper_170"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q25.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q25.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q26.plan.txt index dc978ab9fbae..555bb67d3baa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q26.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q27.plan.txt index 22ec76d44200..ed0e2e73e9f3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q27.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_item_id$gid, s_state$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q28.plan.txt index df987db25c6b..9bf8b7c08e32 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_71"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_129"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q29.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q29.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q30.plan.txt index 111401ec279f..c56b520204f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_91) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_91"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_91) - final aggregation over (ca_state_91, wr_returning_customer_sk_32) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_91", "wr_returning_customer_sk_32"]) - partial aggregation over (ca_state_91, wr_returning_customer_sk_32) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_35"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_91) + final aggregation over (ca_state_91, wr_returning_customer_sk_32) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_91", "wr_returning_customer_sk_32"]) + partial aggregation over (ca_state_91, wr_returning_customer_sk_32) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_35"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_83"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q31.plan.txt index d1d09df16887..efa718f41fb7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_70, d_qoy_42, d_year_38) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_63"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_147", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_147, d_qoy_119, d_year_115) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_147", "d_qoy_119", "d_year_115"]) - partial aggregation over (ca_county_147, d_qoy_119, d_year_115) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_89"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_140"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_147, d_qoy_119, d_year_115) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_147", "d_qoy_119", "d_year_115"]) + partial aggregation over (ca_county_147, d_qoy_119, d_year_115) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_89"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_140"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_289, d_qoy_261, d_year_257) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_289, d_qoy_261, d_year_257) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_221"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_282"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_377", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_377, d_qoy_349, d_year_345) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_377", "d_qoy_349", "d_year_345"]) - partial aggregation over (ca_county_377, d_qoy_349, d_year_345) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_309"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_370"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_201, d_qoy_173, d_year_169) + final aggregation over (ca_county_377, d_qoy_349, d_year_345) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_201", "d_qoy_173", "d_year_169"]) - partial aggregation over (ca_county_201, d_qoy_173, d_year_169) + remote exchange (REPARTITION, HASH, ["ca_county_377", "d_qoy_349", "d_year_345"]) + partial aggregation over (ca_county_377, d_qoy_349, d_year_345) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_309"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_194"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + remote exchange (REPARTITION, HASH, ["ca_address_sk_370"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_201, d_qoy_173, d_year_169) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_201", "d_qoy_173", "d_year_169"]) + partial aggregation over (ca_county_201, d_qoy_173, d_year_169) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_194"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q32.plan.txt index b214e66bcc58..fbb263a8d062 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_21) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_21) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_21) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q33.plan.txt index 0ef86c7948f0..6d9187d8b05b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q33.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_8) + partial aggregation over (i_manufact_id_8) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_25) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_25"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_25) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_25) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_25"]) + partial aggregation over (i_manufact_id_25) + scan item partial aggregation over (i_manufact_id_102) final aggregation over (i_manufact_id_102) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_102"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_102) + partial aggregation over (i_manufact_id_102) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_129) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_129"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_129) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_129"]) + partial aggregation over (i_manufact_id_129) + scan item partial aggregation over (i_manufact_id_208) final aggregation over (i_manufact_id_208) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_208"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_208) + partial aggregation over (i_manufact_id_208) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_235) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_235"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_235) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_235) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_235"]) + partial aggregation over (i_manufact_id_235) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q34.plan.txt index 861d0f0bd0d8..8b6f823e1fe3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q34.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q35.plan.txt index b7f586b43c2d..f1fed4317214 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q35.plan.txt @@ -13,43 +13,37 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q36.plan.txt index eb941375d1fb..3fe86fe51161 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q36.plan.txt @@ -5,18 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q37.plan.txt index 1cad16277680..2582d9fa4122 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q38.plan.txt index fb7e4491f3bb..6590f3356d80 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_7, c_last_name_8, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_54, c_last_name_55, d_date_17) final aggregation over (c_first_name_54, c_last_name_55, d_date_17) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_54, c_last_name_55, d_date_17) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_46"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_109, c_last_name_110, d_date_72) final aggregation over (c_first_name_109, c_last_name_110, d_date_72) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_109, c_last_name_110, d_date_72) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q39.plan.txt index 42f59e4472a9..c06e49455217 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_12", "inv_warehouse_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_68, inv_item_sk_12, inv_warehouse_sk_13, w_warehouse_name_45) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_68", "inv_item_sk_12", "inv_warehouse_sk_13", "w_warehouse_name_45"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_68, inv_item_sk_12, inv_warehouse_sk_13, w_warehouse_name_45) + final aggregation over (d_moy_68, inv_item_sk_12, inv_warehouse_sk_13, w_warehouse_name_45) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_68", "inv_item_sk_12", "inv_warehouse_sk_13", "w_warehouse_name_45"]) + partial aggregation over (d_moy_68, inv_item_sk_12, inv_warehouse_sk_13, w_warehouse_name_45) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q40.plan.txt index 62288a2c3dc5..266142fc4c1a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q42.plan.txt index aaf625b30700..87792b40cd98 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q44.plan.txt index 10edcc19a124..093a41b0ae05 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_7) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) + partial aggregation over (ss_store_sk_7) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_83"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_29) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) + partial aggregation over (ss_item_sk_29) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_7) + final aggregation over (ss_store_sk_62) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_7) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_29) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_29) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_62) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_62"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_62) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_62"]) + partial aggregation over (ss_store_sk_62) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q45.plan.txt index 3104679c2c3b..6af76139314e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q45.plan.txt @@ -8,22 +8,19 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q46.plan.txt index cda34047522b..87414d60d6a0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q46.plan.txt @@ -2,37 +2,32 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q47.plan.txt index 7c9fbe88943e..626ebbc4e55f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_17", "i_category_21", "s_company_name_108", "s_store_name_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_68, d_year_66, i_brand_17, i_category_21, s_company_name_108, s_store_name_96) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_68", "d_year_66", "i_brand_17", "i_category_21", "s_company_name_108", "s_store_name_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_68, d_year_66, i_brand_17, i_category_21, s_company_name_108, s_store_name_96) + final aggregation over (d_moy_68, d_year_66, i_brand_17, i_category_21, s_company_name_108, s_store_name_96) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_68", "d_year_66", "i_brand_17", "i_category_21", "s_company_name_108", "s_store_name_96"]) + partial aggregation over (d_moy_68, d_year_66, i_brand_17, i_category_21, s_company_name_108, s_store_name_96) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_140", "i_category_144", "s_company_name_231", "s_store_name_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_191, d_year_189, i_brand_140, i_category_144, s_company_name_231, s_store_name_219) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_191", "d_year_189", "i_brand_140", "i_category_144", "s_company_name_231", "s_store_name_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_191, d_year_189, i_brand_140, i_category_144, s_company_name_231, s_store_name_219) + final aggregation over (d_moy_191, d_year_189, i_brand_140, i_category_144, s_company_name_231, s_store_name_219) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_191", "d_year_189", "i_brand_140", "i_category_144", "s_company_name_231", "s_store_name_219"]) + partial aggregation over (d_moy_191, d_year_189, i_brand_140, i_category_144, s_company_name_231, s_store_name_219) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q49.plan.txt index 7afb18291ac0..fed42e3537db 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q49.plan.txt @@ -3,65 +3,56 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_19", "expr_23", "rank", "rank_21", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_19, expr_23, rank, rank_21, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - partial aggregation over (wr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_19, expr_23, rank, rank_21, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_71", "expr_75", "rank_73", "rank_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_71, expr_75, rank_73, rank_74) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - partial aggregation over (cr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_71, expr_75, rank_73, rank_74) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_123", "expr_128", "rank_125", "rank_126", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_123, expr_128, rank_125, rank_126, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - partial aggregation over (sr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_123, expr_128, rank_125, rank_126, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q50.plan.txt index c28e1f11bc9a..c2a9d481bf10 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q50.plan.txt @@ -8,16 +8,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q51.plan.txt index bed68d8668d2..5020827b6750 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_9, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_9", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_9, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_9, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_9", "ss_item_sk"]) + partial aggregation over (d_date_9, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q52.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q53.plan.txt index 2206a00753f9..cd10ab3698e6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q53.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy, i_manufact_id) + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q54.plan.txt index 1e79f1eef876..90dc47e1a40d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_114) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_114"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_114) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_114) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_114"]) + partial aggregation over (expr_114) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q55.plan.txt index 92c0bba58211..f89f2f078b63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q56.plan.txt index 9cda6612750d..be881ce912e9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q56.plan.txt @@ -6,30 +6,28 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_8) + partial aggregation over (i_item_id_8) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_13) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_13"]) + partial aggregation over (i_item_id_13) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address partial aggregation over (i_item_id_90) final aggregation over (i_item_id_90) local exchange (GATHER, SINGLE, []) @@ -37,29 +35,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_90) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_117) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_117"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_117) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_117) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_117"]) + partial aggregation over (i_item_id_117) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_196) final aggregation over (i_item_id_196) local exchange (GATHER, SINGLE, []) @@ -67,26 +62,23 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_196) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_223) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_223"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_223) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_223) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_223"]) + partial aggregation over (i_item_id_223) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_179"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q57.plan.txt index 23e9d7279988..e9f0b1bd10c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_108", "i_brand_17", "i_category_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_108, d_moy_79, d_year_77, i_brand_17, i_category_21) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_108", "d_moy_79", "d_year_77", "i_brand_17", "i_category_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_108, d_moy_79, d_year_77, i_brand_17, i_category_21) + final aggregation over (cc_name_108, d_moy_79, d_year_77, i_brand_17, i_category_21) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_108", "d_moy_79", "d_year_77", "i_brand_17", "i_category_21"]) + partial aggregation over (cc_name_108, d_moy_79, d_year_77, i_brand_17, i_category_21) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_243", "i_brand_152", "i_category_156"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_243, d_moy_214, d_year_212, i_brand_152, i_category_156) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_243", "d_moy_214", "d_year_212", "i_brand_152", "i_category_156"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_243, d_moy_214, d_year_212, i_brand_152, i_category_156) + final aggregation over (cc_name_243, d_moy_214, d_year_212, i_brand_152, i_category_156) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_243", "d_moy_214", "d_year_212", "i_brand_152", "i_category_156"]) + partial aggregation over (cc_name_243, d_moy_214, d_year_212, i_brand_152, i_category_156) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q58.plan.txt index 2aa7dc85d1da..b5044356bbce 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q58.plan.txt @@ -4,32 +4,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_8) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_8) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_8) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_8"]) + partial aggregation over (d_date_8) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -37,59 +35,53 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_133) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_133) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_133) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_133"]) + partial aggregation over (d_date_133) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_201) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_201"]) partial aggregation over (i_item_id_201) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_260) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_260"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_260) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_260) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_260"]) + partial aggregation over (d_date_260) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q59.plan.txt index b652ec89d3db..8c0032f32687 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q59.plan.txt @@ -4,53 +4,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_210", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_56"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_79, ss_store_sk_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_79", "ss_store_sk_56"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_79, ss_store_sk_56) - final aggregation over (d_day_name_89, d_week_seq_79, ss_store_sk_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_89", "d_week_seq_79", "ss_store_sk_56"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_89, d_week_seq_79, ss_store_sk_56) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_79, ss_store_sk_56) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_79", "ss_store_sk_56"]) + partial aggregation over (d_week_seq_79, ss_store_sk_56) + final aggregation over (d_day_name_89, d_week_seq_79, ss_store_sk_56) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_89", "d_week_seq_79", "ss_store_sk_56"]) + partial aggregation over (d_day_name_89, d_week_seq_79, ss_store_sk_56) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_sk_122"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_123"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q60.plan.txt index fe3367ade51d..b42dce272fe4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q60.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_8) + partial aggregation over (i_item_id_8) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_13) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_13"]) + partial aggregation over (i_item_id_13) + scan item partial aggregation over (i_item_id_90) final aggregation over (i_item_id_90) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_90) + partial aggregation over (i_item_id_90) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_117) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_117"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_117) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_117) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_117"]) + partial aggregation over (i_item_id_117) + scan item partial aggregation over (i_item_id_196) final aggregation over (i_item_id_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_196) + partial aggregation over (i_item_id_196) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_223) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_223"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_223) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_223) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_223"]) + partial aggregation over (i_item_id_223) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q61.plan.txt index a466d09e6f9c..aa366b079da7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q61.plan.txt @@ -6,28 +6,26 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan promotion @@ -37,25 +35,23 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_21"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_107"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q62.plan.txt index a00a28581512..61172c74fc4f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q63.plan.txt index 1732f85548b8..23e28eddc4b7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q63.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_manager_id) + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q64.plan.txt index e88e3b4f2ce0..6497c0e5dec4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q64.plan.txt @@ -3,195 +3,177 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_119, ca_street_name, ca_street_name_116, ca_street_number, ca_street_number_115, ca_zip, ca_zip_122, d_year, d_year_19, d_year_50, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_119", "ca_street_name", "ca_street_name_116", "ca_street_number", "ca_street_number_115", "ca_zip", "ca_zip_122", "d_year", "d_year_19", "d_year_50", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - partial aggregation over (ca_city, ca_city_119, ca_street_name, ca_street_name_116, ca_street_number, ca_street_number_115, ca_zip, ca_zip_122, d_year, d_year_19, d_year_50, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_113"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ca_city_119, ca_street_name, ca_street_name_116, ca_street_number, ca_street_number_115, ca_zip, ca_zip_122, d_year, d_year_19, d_year_50, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_119", "ca_street_name", "ca_street_name_116", "ca_street_number", "ca_street_number_115", "ca_zip", "ca_zip_122", "d_year", "d_year_19", "d_year_50", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_119, ca_street_name, ca_street_name_116, ca_street_number, ca_street_number_115, ca_zip, ca_zip_122, d_year, d_year_19, d_year_50, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_113"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_84"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_name_368", "s_zip_388", "ss_item_sk_152"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_484, ca_city_500, ca_street_name_481, ca_street_name_497, ca_street_number_480, ca_street_number_496, ca_zip_487, ca_zip_503, d_year_276, d_year_307, d_year_338, i_product_name_543, s_store_name_368, s_zip_388, ss_item_sk_152) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_484", "ca_city_500", "ca_street_name_481", "ca_street_name_497", "ca_street_number_480", "ca_street_number_496", "ca_zip_487", "ca_zip_503", "d_year_276", "d_year_307", "d_year_338", "i_product_name_543", "s_store_name_368", "s_zip_388", "ss_item_sk_152"]) - partial aggregation over (ca_city_484, ca_city_500, ca_street_name_481, ca_street_name_497, ca_street_number_480, ca_street_number_496, ca_zip_487, ca_zip_503, d_year_276, d_year_307, d_year_338, i_product_name_543, s_store_name_368, s_zip_388, ss_item_sk_152) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_494"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city_484, ca_city_500, ca_street_name_481, ca_street_name_497, ca_street_number_480, ca_street_number_496, ca_zip_487, ca_zip_503, d_year_276, d_year_307, d_year_338, i_product_name_543, s_store_name_368, s_zip_388, ss_item_sk_152) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_484", "ca_city_500", "ca_street_name_481", "ca_street_name_497", "ca_street_number_480", "ca_street_number_496", "ca_zip_487", "ca_zip_503", "d_year_276", "d_year_307", "d_year_338", "i_product_name_543", "s_store_name_368", "s_zip_388", "ss_item_sk_152"]) + partial aggregation over (ca_city_484, ca_city_500, ca_street_name_481, ca_street_name_497, ca_street_number_480, ca_street_number_496, ca_zip_487, ca_zip_503, d_year_276, d_year_307, d_year_338, i_product_name_543, s_store_name_368, s_zip_388, ss_item_sk_152) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_494"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk_399"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk_399"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_397"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_153"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_397"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_153"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_152", "ss_ticket_number_159"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk_152", "ss_ticket_number_159"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_178", "sr_ticket_number_185"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_214) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_214"]) - partial aggregation over (cs_item_sk_214) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_214", "cs_order_number_216"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_238", "cr_order_number_252"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk_178", "sr_ticket_number_185"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_395"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk_214) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk_214"]) + partial aggregation over (cs_item_sk_214) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_214", "cs_order_number_216"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_238", "cr_order_number_252"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_395"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_428"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_428"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q65.plan.txt index 197f7ce1d450..16d4126e5149 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_11, ss_store_sk_16) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_11", "ss_store_sk_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_11, ss_store_sk_16) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_11, ss_store_sk_16) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q66.plan.txt index 5796c6a13754..d78688a29a6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_381, d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) final aggregation over (d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_90", "w_city_75", "w_country_79", "w_county_76", "w_state_77", "w_warehouse_name_69", "w_warehouse_sq_ft_70"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) - final aggregation over (d_moy_92, d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_92", "d_year_90", "w_city_75", "w_country_79", "w_county_76", "w_state_77", "w_warehouse_name_69", "w_warehouse_sq_ft_70"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_92, d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) + partial aggregation over (d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) + final aggregation over (d_moy_92, d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_92", "d_year_90", "w_city_75", "w_country_79", "w_county_76", "w_state_77", "w_warehouse_name_69", "w_warehouse_sq_ft_70"]) + partial aggregation over (d_moy_92, d_year_90, w_city_75, w_country_79, w_county_76, w_state_77, w_warehouse_name_69, w_warehouse_sq_ft_70) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q68.plan.txt index b6745add56f9..ecbd5a004789 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q68.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q69.plan.txt index bc516ac6a348..d304792469c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q69.plan.txt @@ -3,55 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (LEFT, PARTITIONED): - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - final aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (LEFT, PARTITIONED): + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q70.plan.txt index cd2a781d1350..ad464d39cb92 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_97) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_56) final aggregation over (s_state_56) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_56"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_56) + partial aggregation over (s_state_56) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q72.plan.txt index 4417571368bb..9da6042a0c9d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q72.plan.txt @@ -9,37 +9,34 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -48,5 +45,4 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q73.plan.txt index 9fa35d4ee8c7..30d700ed0193 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q73.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q74.plan.txt index 85b74bb40480..213bb1464a96 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q74.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_74, c_first_name_81, c_last_name_82, d_year_126) + final aggregation over (c_customer_id_74, c_first_name_81, c_last_name_82, d_year_126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_74", "c_first_name_81", "c_last_name_82", "d_year_126"]) + partial aggregation over (c_customer_id_74, c_first_name_81, c_last_name_82, d_year_126) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_97"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_73"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_343"]) + final aggregation over (c_customer_id_343, c_first_name_350, c_last_name_351, d_year_406) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_74", "c_first_name_81", "c_last_name_82", "d_year_126"]) - partial aggregation over (c_customer_id_74, c_first_name_81, c_last_name_82, d_year_126) + remote exchange (REPARTITION, HASH, ["c_customer_id_343", "c_first_name_350", "c_last_name_351", "d_year_406"]) + partial aggregation over (c_customer_id_343, c_first_name_350, c_last_name_351, d_year_406) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_367"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_343"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_343, c_first_name_350, c_last_name_351, d_year_406) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_343", "c_first_name_350", "c_last_name_351", "d_year_406"]) - partial aggregation over (c_customer_id_343, c_first_name_350, c_last_name_351, d_year_406) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_367"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_342"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_342"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_528"]) + final aggregation over (c_customer_id_528, c_first_name_535, c_last_name_536, d_year_591) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + remote exchange (REPARTITION, HASH, ["c_customer_id_528", "c_first_name_535", "c_last_name_536", "d_year_591"]) + partial aggregation over (c_customer_id_528, c_first_name_535, c_last_name_536, d_year_591) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_552"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_528"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_528, c_first_name_535, c_last_name_536, d_year_591) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_528", "c_first_name_535", "c_last_name_536", "d_year_591"]) - partial aggregation over (c_customer_id_528, c_first_name_535, c_last_name_536, d_year_591) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_552"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_527"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_527"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q75.plan.txt index 904e76f040c1..d5a331127722 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_14, expr, expr_21, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_14, expr, expr_21, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_32", "i_category_id_36", "i_class_id_34", "i_manufact_id_38"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_56, expr_84, expr_85, i_brand_id_32, i_category_id_36, i_class_id_34, i_manufact_id_38) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_56, expr_84, expr_85, i_brand_id_32, i_category_id_36, i_class_id_34, i_manufact_id_38) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_96", "i_category_id_100", "i_class_id_98", "i_manufact_id_102"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_120, expr_148, expr_149, i_brand_id_96, i_category_id_100, i_class_id_98, i_manufact_id_102) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_120, expr_148, expr_149, i_brand_id_96, i_category_id_100, i_class_id_98, i_manufact_id_102) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (d_year_152, i_brand_id_153, i_category_id_155, i_class_id_154, i_manufact_id_156) final aggregation over (d_year_152, i_brand_id_153, i_category_id_155, i_class_id_154, i_manufact_id_156, sales_amt_158, sales_cnt_157) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_210", "i_category_id_214", "i_class_id_212", "i_manufact_id_216"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_234, expr_289, expr_290, i_brand_id_210, i_category_id_214, i_class_id_212, i_manufact_id_216) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_261", "cr_order_number_275"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_181", "cs_order_number_183"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_234, expr_289, expr_290, i_brand_id_210, i_category_id_214, i_class_id_212, i_manufact_id_216) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_261", "cr_order_number_275"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_181", "cs_order_number_183"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_324", "i_category_id_328", "i_class_id_326", "i_manufact_id_330"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_348, expr_396, expr_397, i_brand_id_324, i_category_id_328, i_class_id_326, i_manufact_id_330) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_375", "sr_ticket_number_382"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_293", "ss_ticket_number_300"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_348, expr_396, expr_397, i_brand_id_324, i_category_id_328, i_class_id_326, i_manufact_id_330) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_375", "sr_ticket_number_382"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_293", "ss_ticket_number_300"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_442", "i_category_id_446", "i_class_id_444", "i_manufact_id_448"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_466, expr_518, expr_519, i_brand_id_442, i_category_id_446, i_class_id_444, i_manufact_id_448) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_493", "wr_order_number_504"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_401", "ws_order_number_415"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_466, expr_518, expr_519, i_brand_id_442, i_category_id_446, i_class_id_444, i_manufact_id_448) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_493", "wr_order_number_504"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_401", "ws_order_number_415"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q76.plan.txt index a946c27d3253..2c646494d11f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q76.plan.txt @@ -3,42 +3,37 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_11", "d_year_10", "expr_142", "expr_143", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_11, d_year_10, expr_142, expr_143, i_category_6) + partial aggregation over (d_qoy_11, d_year_10, expr_142, expr_143, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["d_qoy_55", "d_year_51", "expr_148", "expr_149", "i_category_32"]) partial aggregation over (d_qoy_55, d_year_51, expr_148, expr_149, i_category_32) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_45"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_118", "d_year_114", "expr_139", "expr_141", "i_category_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_118, d_year_114, expr_139, expr_141, i_category_95) + partial aggregation over (d_qoy_118, d_year_114, expr_139, expr_141, i_category_95) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q78.plan.txt index 0353497d6cec..8644def442f8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q78.plan.txt @@ -2,56 +2,48 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_year, ss_customer_sk, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + join (INNER, PARTITIONED): + final aggregation over (d_year, ss_customer_sk, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) + partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (d_year_21, ws_bill_customer_sk, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year_21", "ws_bill_customer_sk", "ws_item_sk"]) - partial aggregation over (d_year_21, ws_bill_customer_sk, ws_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + final aggregation over (d_year_21, ws_bill_customer_sk, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year_21", "ws_bill_customer_sk", "ws_item_sk"]) + partial aggregation over (d_year_21, ws_bill_customer_sk, ws_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_63) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_63"]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_63) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_63) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_63"]) + partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_63) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q81.plan.txt index 0221aac3009a..0f6856f9c813 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q81.plan.txt @@ -9,16 +9,14 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state, cr_returning_customer_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -31,24 +29,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state_94) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_94) - final aggregation over (ca_state_94, cr_returning_customer_sk_32) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_94", "cr_returning_customer_sk_32"]) - partial aggregation over (ca_state_94, cr_returning_customer_sk_32) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_35"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_86"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_94) + final aggregation over (ca_state_94, cr_returning_customer_sk_32) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_94", "cr_returning_customer_sk_32"]) + partial aggregation over (ca_state_94, cr_returning_customer_sk_32) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_35"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_86"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q82.plan.txt index 4232c710d2d6..532952036fc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q83.plan.txt index fbc09e235660..cc4b500d368e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q83.plan.txt @@ -7,34 +7,30 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_8) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_8) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_43) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_43"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_43) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_8) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_8"]) + partial aggregation over (d_date_8) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_43) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_43"]) + partial aggregation over (d_week_seq_43) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_79) local exchange (GATHER, SINGLE, []) @@ -42,65 +38,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_79) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_138) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_138"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_138) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_173) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_173"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_173) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_138) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_138"]) + partial aggregation over (d_date_138) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_173) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_173"]) + partial aggregation over (d_week_seq_173) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_211) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_211"]) partial aggregation over (i_item_id_211) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_270) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_270"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_270) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_305) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_305"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_305) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_270) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_270"]) + partial aggregation over (d_date_270) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_305) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_305"]) + partial aggregation over (d_week_seq_305) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q85.plan.txt index 3c7c4fb1c2f8..ab3b2c159101 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q85.plan.txt @@ -3,41 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q86.plan.txt index f880adf8bf2d..990089879464 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q87.plan.txt index fb7e4491f3bb..6590f3356d80 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_7, c_last_name_8, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_54, c_last_name_55, d_date_17) final aggregation over (c_first_name_54, c_last_name_55, d_date_17) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_54, c_last_name_55, d_date_17) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_46"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_109, c_last_name_110, d_date_72) final aggregation over (c_first_name_109, c_last_name_110, d_date_72) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_109, c_last_name_110, d_date_72) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q89.plan.txt index b5495ae3801b..396d705bbc38 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q91.plan.txt index 3427cfc56e6c..f1c8dfe5de34 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q91.plan.txt @@ -9,26 +9,23 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q92.plan.txt index f7f467b476d8..2e92e9b3a1a9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_9) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_9) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_9) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q94.plan.txt index e643c7b023cc..217dddd8d551 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q94.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_26, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_26", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_26, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_26, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_26", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) + partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_26, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q95.plan.txt index 956e3f99badd..9daf91528a0a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q95.plan.txt @@ -9,41 +9,35 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_107"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_144"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales join (INNER, PARTITIONED): final aggregation over (ws_order_number_26) local exchange (GATHER, SINGLE, []) partial aggregation over (ws_order_number_26) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_63"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q98.plan.txt index e5840c437809..9775b6c511f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q98.plan.txt @@ -3,18 +3,16 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q99.plan.txt index e17b712a1894..6e2c1451c321 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q01.plan.txt index 630021f6984d..d091b5f71d8b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q01.plan.txt @@ -4,42 +4,37 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (sr_store_sk_12) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_12) - final aggregation over (sr_customer_sk_8, sr_store_sk_12) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_8", "sr_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_8, sr_store_sk_12) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_12) + final aggregation over (sr_customer_sk_8, sr_store_sk_12) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_8", "sr_store_sk_12"]) + partial aggregation over (sr_customer_sk_8, sr_store_sk_12) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q02.plan.txt index a9e4ac5bcfad..f26bc3efa106 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_124) - final aggregation over (d_day_name_134, d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) - partial aggregation over (d_day_name_134, d_week_seq_124) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_124) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) + partial aggregation over (d_week_seq_124) + final aggregation over (d_day_name_134, d_week_seq_124) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) + partial aggregation over (d_day_name_134, d_week_seq_124) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q04.plan.txt index 954de084f516..84c91be8eac4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q04.plan.txt @@ -4,117 +4,99 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) + partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_869"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) + final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) - partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) + partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_869"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1567"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) - partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1567"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) - partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_569"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) - partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1267"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) - partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) + partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_183"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_569"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) + final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) + partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1267"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) + final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) + partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_183"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q05.plan.txt index da6352ecc72f..169f66124bda 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_81", "ws_order_number_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q06.plan.txt index 1c7ee06ff527..fb3592d65b6e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q06.plan.txt @@ -11,31 +11,27 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_11) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) + partial aggregation over (d_month_seq_11) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -44,9 +40,8 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_50) - scan item + partial aggregation over (i_category_50) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q07.plan.txt index d209bd09ae61..32f4763b443d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q07.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q08.plan.txt index 998b0b959fcb..45b29afc325b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_35"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_26) - scan customer_address + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_29) - final aggregation over (ca_zip_16) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_16"]) - partial aggregation over (ca_zip_16) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_29) + final aggregation over (ca_zip_16) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_16"]) + partial aggregation over (ca_zip_16) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q10.plan.txt index a53dd7834678..7d60d2a5fdd5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q10.plan.txt @@ -3,54 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q11.plan.txt index 6fee2f7f3db5..668a1795f1c6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q11.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) + partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_100"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) + final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) - partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) + partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_380"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) - partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_380"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) + final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) + partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_573"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) - partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_573"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q12.plan.txt index 609e3537018c..eb114fe9092d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q12.plan.txt @@ -2,20 +2,17 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q13.plan.txt index 30470310dca0..a4d7d7d75438 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q13.plan.txt @@ -8,23 +8,20 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q14.plan.txt index 21eb6df883d1..c8326b25e125 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) - partial aggregation over (i_item_sk_13) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) + partial aggregation over (i_item_sk_13) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) + partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) + partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) + partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_469", "i_category_id_473", "i_class_id_471"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_518) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) - partial aggregation over (i_item_sk_518) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_542, category_id_544, class_id_543) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) + scan item + final aggregation over (i_item_sk_518) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) + partial aggregation over (i_item_sk_518) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) + scan item + final aggregation over (brand_id_542, category_id_544, class_id_543) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) + partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) + partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) + partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -183,66 +171,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1052", "i_category_id_1056", "i_class_id_1054"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_1101) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) - partial aggregation over (i_item_sk_1101) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_1125, category_id_1127, class_id_1126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) + scan item + final aggregation over (i_item_sk_1101) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) + partial aggregation over (i_item_sk_1101) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) + scan item + final aggregation over (brand_id_1125, category_id_1127, class_id_1126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) + partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) + partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) + partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q15.plan.txt index b22f9d12537c..56302f05c739 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q16.plan.txt index 67c9ba77fd3d..2c79deb552b5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q16.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (cr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_22", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_22", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q17.plan.txt index 7a9af86c0c6c..49c98c0b8dda 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q17.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_state) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q18.plan.txt index 7f9cbb3ebf63..5f2dfaaeb4fa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q18.plan.txt @@ -4,35 +4,33 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) remote exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) partial aggregation over (ca_country$gid, ca_county$gid, ca_state$gid, groupid, i_item_id$gid) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) + remote exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) + scan customer_demographics local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q19.plan.txt index 967aa874de69..c500acf6fca3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q19.plan.txt @@ -7,26 +7,23 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q20.plan.txt index 2dce6eadbb48..852a9d2a3e5c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q20.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q21.plan.txt index e8067ff5be06..50e40917c430 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q23.plan.txt index 64f6afa02051..a98a247c3340 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_6, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_6, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_42) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_42) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_42) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_70) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_70"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_93"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_184) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_184) - final aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_184", "substr$gid_265"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_184) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_184"]) + partial aggregation over (ss_item_sk_184) + final aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_184", "substr$gid_265"]) + partial aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_273) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_273) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_273) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_273"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_321) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_321"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_344"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q24.plan.txt index 791a3935f4a9..243f543065be 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q24.plan.txt @@ -5,40 +5,35 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,23 +52,19 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_12", "ss_ticket_number_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_37", "sr_ticket_number_44"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_113"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_142", "upper_159"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q25.plan.txt index 4dcfe955597d..8eedb8e05888 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q25.plan.txt @@ -7,35 +7,31 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q26.plan.txt index dc978ab9fbae..555bb67d3baa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q26.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q27.plan.txt index 22ec76d44200..b637b1d27f97 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q27.plan.txt @@ -4,23 +4,23 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, HASH, ["groupid", "i_item_id$gid", "s_state$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_item_id$gid", "s_state$gid"]) partial aggregation over (groupid, i_item_id$gid, s_state$gid) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, HASH, ["i_item_id"]) + remote exchange (REPARTITION, HASH, ["i_item_id"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q28.plan.txt index f64d5530bbf2..069ba6e93218 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q29.plan.txt index 4dcfe955597d..8eedb8e05888 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q29.plan.txt @@ -7,35 +7,31 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q30.plan.txt index f2d3b05fe566..4db8148f3ed6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_85) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_85"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_85) - final aggregation over (ca_state_85, wr_returning_customer_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_27"]) - partial aggregation over (ca_state_85, wr_returning_customer_sk_27) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_30"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ca_state_85) + final aggregation over (ca_state_85, wr_returning_customer_sk_27) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_27"]) + partial aggregation over (ca_state_85, wr_returning_customer_sk_27) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_30"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q31.plan.txt index c65ae2ca3c5a..04a7a0fc2cd4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_66, d_qoy_39, d_year_35) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_140, d_qoy_113, d_year_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) - partial aggregation over (ca_county_140, d_qoy_113, d_year_109) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_140, d_qoy_113, d_year_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) + partial aggregation over (ca_county_140, d_qoy_113, d_year_109) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_83"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_276, d_qoy_249, d_year_245) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_269"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_294"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_191, d_qoy_164, d_year_160) + final aggregation over (ca_county_361, d_qoy_334, d_year_330) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) - partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) + partial aggregation over (ca_county_361, d_qoy_334, d_year_330) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_294"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_191, d_qoy_164, d_year_160) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) + partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q32.plan.txt index be2a74f9a07a..e09ec6de273c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_18) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_18) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_18) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q33.plan.txt index 2502d90464f0..1b2427d4e654 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q33.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_6) + partial aggregation over (i_manufact_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_22) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_22) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) + partial aggregation over (i_manufact_id_22) + scan item partial aggregation over (i_manufact_id_95) final aggregation over (i_manufact_id_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_95) + partial aggregation over (i_manufact_id_95) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_121) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_121) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_121) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) + partial aggregation over (i_manufact_id_121) + scan item partial aggregation over (i_manufact_id_196) final aggregation over (i_manufact_id_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_196) + partial aggregation over (i_manufact_id_196) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_222) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_222) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_222) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) + partial aggregation over (i_manufact_id_222) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q34.plan.txt index 861d0f0bd0d8..8b6f823e1fe3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q34.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q35.plan.txt index 8353da6955f2..f94ec8be8411 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q35.plan.txt @@ -11,45 +11,39 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q36.plan.txt index c7aa800fe3c9..d640f5ac0d14 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q36.plan.txt @@ -5,18 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q37.plan.txt index 1cad16277680..2582d9fa4122 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q38.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q39.plan.txt index b74a3c77c64a..eb2349ef1574 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_8", "inv_warehouse_sk_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_8", "inv_warehouse_sk_9", "w_warehouse_name_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + final aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_8", "inv_warehouse_sk_9", "w_warehouse_name_40"]) + partial aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q40.plan.txt index 62288a2c3dc5..266142fc4c1a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q42.plan.txt index d6a3c8cac82b..17cc12f10e9e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q44.plan.txt index f8aeb77ec755..7cd4ca5a0807 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) + partial aggregation over (ss_store_sk_6) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_80"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_27) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_27"]) + partial aggregation over (ss_item_sk_27) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_6) + final aggregation over (ss_store_sk_59) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_6) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_27"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_27) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_59) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_59) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_59"]) + partial aggregation over (ss_store_sk_59) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q45.plan.txt index b275b76606e4..5b30cdf7426c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q45.plan.txt @@ -10,23 +10,20 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q46.plan.txt index 6ddeddd43120..e8aa3db7886c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q46.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q47.plan.txt index 93faff459e9c..2d63bde52f55 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) + partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) + partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q48.plan.txt index 4e0866c981c0..7eb552bd09e3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q48.plan.txt @@ -7,20 +7,17 @@ final aggregation over () remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q49.plan.txt index 66381314bec4..6f687392953d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q49.plan.txt @@ -3,65 +3,56 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_17", "expr_21", "rank", "rank_19", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - partial aggregation over (wr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - partial aggregation over (cr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - partial aggregation over (sr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q50.plan.txt index c28e1f11bc9a..c2a9d481bf10 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q50.plan.txt @@ -8,16 +8,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q51.plan.txt index 06b2887fa30d..ed5e8607b930 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_7, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_7, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_7, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) + partial aggregation over (d_date_7, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q52.plan.txt index ab4fb497ba27..33752e693e6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q53.plan.txt index 2206a00753f9..cd10ab3698e6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q53.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy, i_manufact_id) + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q54.plan.txt index 73276be84ef1..23ee4dacf66b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_104) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_104) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_104) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_104"]) + partial aggregation over (expr_104) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q55.plan.txt index ad84f78b05f8..0322599f01e5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q56.plan.txt index b5c953a4d97e..e96c8c31aabb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q56.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q57.plan.txt index 25bb57998645..26f16c8477dc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) + partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) + partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q58.plan.txt index 7bd45a0070f2..192c6d359edd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q58.plan.txt @@ -7,31 +7,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_69) local exchange (GATHER, SINGLE, []) @@ -39,59 +36,53 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_69) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_126"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_126) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_126"]) + partial aggregation over (d_date_126) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_191) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_191"]) partial aggregation over (i_item_id_191) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_248) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_248"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_248) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_248) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_248"]) + partial aggregation over (d_date_248) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_190"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q59.plan.txt index 133dd26321d0..7f584558007b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q59.plan.txt @@ -4,53 +4,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_52) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_75, ss_store_sk_52) - final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_75, ss_store_sk_52) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_52"]) + partial aggregation over (d_week_seq_75, ss_store_sk_52) + final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_52"]) + partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_sk_117"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q60.plan.txt index 70f98c7eb54e..4616b0ad95b6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q60.plan.txt @@ -6,83 +6,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_6"]) + scan item + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_83"]) + scan item + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_184"]) + scan item + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q61.plan.txt index 9205d0d586d7..ff3d350a643f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q61.plan.txt @@ -6,28 +6,26 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan promotion @@ -37,25 +35,23 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q62.plan.txt index c66895aac6b8..90f04a71a65b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan ship_mode diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q63.plan.txt index 1732f85548b8..23e28eddc4b7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q63.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_manager_id) + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q64.plan.txt index a133a52c71bb..5b7e66dd9c00 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q64.plan.txt @@ -3,94 +3,166 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_133"]) + final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_133"]) + partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_135"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_134"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk_133", "ss_ticket_number_140"]) join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk_158", "sr_ticket_number_165"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk_193) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_193"]) + partial aggregation over (cs_item_sk_193) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_193", "cs_order_number_195"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_216", "cr_order_number_230"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -98,100 +170,10 @@ remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_135"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_134"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_133", "ss_ticket_number_140"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_158", "sr_ticket_number_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_193) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_193"]) - partial aggregation over (cs_item_sk_193) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_193", "cs_order_number_195"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_216", "cr_order_number_230"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q65.plan.txt index 16047fe4c95e..0c80486a2845 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_7, ss_store_sk_12) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_7", "ss_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_7, ss_store_sk_12) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_7, ss_store_sk_12) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q66.plan.txt index aa228d59f437..b87ae78025f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_372, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) final aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) + partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q68.plan.txt index 52aa91be911d..5feab810ee06 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q68.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q69.plan.txt index 8de8d86fbc94..4b87cab3294e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q69.plan.txt @@ -7,47 +7,42 @@ local exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q70.plan.txt index 542ab85591d7..0ba93430749a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_53) final aggregation over (s_state_53) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_53) + partial aggregation over (s_state_53) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q72.plan.txt index bdbdae78e9d9..9c47f45ae5a1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q72.plan.txt @@ -9,37 +9,34 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -48,5 +45,4 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q73.plan.txt index 9fa35d4ee8c7..30d700ed0193 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q73.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q74.plan.txt index d9e31e2a2fb7..1faae3d6738a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q74.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) + partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_90"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) + final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) - partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) + partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_351"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) - partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_351"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) + final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) + partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_530"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) - partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_530"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q75.plan.txt index cb8fd860379c..72ac1ec6cf65 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145) final aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145, sales_amt_147, sales_cnt_146) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_198", "i_category_id_202", "i_class_id_200", "i_manufact_id_204"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_246", "cr_order_number_260"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_169", "cs_order_number_171"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_246", "cr_order_number_260"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_169", "cs_order_number_171"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_356", "sr_ticket_number_363"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_277", "ss_ticket_number_284"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_356", "sr_ticket_number_363"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_277", "ss_ticket_number_284"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_470", "wr_order_number_481"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_381", "ws_order_number_395"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_470", "wr_order_number_481"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_381", "ws_order_number_395"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q76.plan.txt index c1d7f408a238..96f6228260dc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q76.plan.txt @@ -3,44 +3,38 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_10", "d_year_9", "expr_134", "expr_135", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["d_qoy_51", "d_year_47", "expr_140", "expr_141", "i_category_29"]) partial aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_111", "d_year_107", "expr_131", "expr_133", "i_category_89"]) partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q78.plan.txt index 4f6eec106eaa..50161185b413 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q78.plan.txt @@ -2,56 +2,48 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_year, ss_customer_sk, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + join (INNER, PARTITIONED): + final aggregation over (d_year, ss_customer_sk, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) + partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) - partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) + partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) + partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q81.plan.txt index 45022024d774..5011246b8efd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q81.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) - partial aggregation over (ca_state, cr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_state, cr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) + partial aggregation over (ca_state, cr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_88) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_88"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_88) - final aggregation over (ca_state_88, cr_returning_customer_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_27"]) - partial aggregation over (ca_state_88, cr_returning_customer_sk_27) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_30"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_88) + final aggregation over (ca_state_88, cr_returning_customer_sk_27) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_27"]) + partial aggregation over (ca_state_88, cr_returning_customer_sk_27) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_30"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q82.plan.txt index 4232c710d2d6..532952036fc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q83.plan.txt index 6023a24c3a48..d5992a1244c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q83.plan.txt @@ -7,34 +7,30 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_40) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) + partial aggregation over (d_week_seq_40) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -42,65 +38,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_131) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_131"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_131) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_165) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_165) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_131) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_131"]) + partial aggregation over (d_date_131) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_165) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) + partial aggregation over (d_week_seq_165) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_201) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_201"]) partial aggregation over (i_item_id_201) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_258) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_258"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_258) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_292) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_292) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_258) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_258"]) + partial aggregation over (d_date_258) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_292) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) + partial aggregation over (d_week_seq_292) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q85.plan.txt index b5198e5c38f0..2a72b1dbbed5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q85.plan.txt @@ -3,41 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_6", "cd_education_status_9", "cd_marital_status_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_education_status", "cd_marital_status", "wr_returning_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_6", "cd_education_status_9", "cd_marital_status_8"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_education_status", "cd_marital_status", "wr_returning_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q86.plan.txt index a3b94e200aa6..cca927926827 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q87.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q89.plan.txt index b5495ae3801b..396d705bbc38 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q91.plan.txt index 2da8cda3eecd..750e4be2dcb9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q91.plan.txt @@ -8,32 +8,28 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q92.plan.txt index d4858936dcb2..2dfed78f9a62 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_6) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_6) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q94.plan.txt index 24b3ed828774..2341a65cbe9b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q94.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_22", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_22", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) + partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q95.plan.txt index b5f1e9ba58e9..5e42003378f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q95.plan.txt @@ -9,41 +9,35 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_136"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales join (INNER, PARTITIONED): final aggregation over (ws_order_number_22) local exchange (GATHER, SINGLE, []) partial aggregation over (ws_order_number_22) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_58"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q98.plan.txt index e5840c437809..9775b6c511f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q98.plan.txt @@ -3,18 +3,16 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q99.plan.txt index 652884fb1181..7f8838ac985a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/partitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q01.plan.txt index 56f3c72ccdcc..114fb89fe851 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q01.plan.txt @@ -4,42 +4,37 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (sr_store_sk_13) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_13) - final aggregation over (sr_customer_sk_9, sr_store_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_9", "sr_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_9, sr_store_sk_13) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_13) + final aggregation over (sr_customer_sk_9, sr_store_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_9", "sr_store_sk_13"]) + partial aggregation over (sr_customer_sk_9, sr_store_sk_13) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q02.plan.txt index a9e4ac5bcfad..f26bc3efa106 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_124) - final aggregation over (d_day_name_134, d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) - partial aggregation over (d_day_name_134, d_week_seq_124) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_124) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) + partial aggregation over (d_week_seq_124) + final aggregation over (d_day_name_134, d_week_seq_124) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) + partial aggregation over (d_day_name_134, d_week_seq_124) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q04.plan.txt index 61aa19348540..56356d683fe6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q04.plan.txt @@ -4,117 +4,99 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) + partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_870"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) + final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) - partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) + partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_870"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1568"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) - partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1568"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) - partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_570"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) - partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1268"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) - partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) + partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_570"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) + final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) + partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1268"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) + final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) + partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_184"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q05.plan.txt index ded90e2ed237..a837004a655b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_82", "ws_order_number_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q06.plan.txt index 1c7ee06ff527..fb3592d65b6e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q06.plan.txt @@ -11,31 +11,27 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_11) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) + partial aggregation over (d_month_seq_11) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -44,9 +40,8 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_50) - scan item + partial aggregation over (i_category_50) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q07.plan.txt index d209bd09ae61..32f4763b443d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q07.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q08.plan.txt index 998b0b959fcb..45b29afc325b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_35"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_26) - scan customer_address + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_29) - final aggregation over (ca_zip_16) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_16"]) - partial aggregation over (ca_zip_16) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_29) + final aggregation over (ca_zip_16) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_16"]) + partial aggregation over (ca_zip_16) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q10.plan.txt index a53dd7834678..7d60d2a5fdd5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q10.plan.txt @@ -3,54 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q11.plan.txt index 9e37382a600e..b0c5b40468d0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q11.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) + partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_101"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) + final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) - partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) + partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_381"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) - partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_381"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) + final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) + partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_574"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) - partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_574"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q12.plan.txt index 609e3537018c..eb114fe9092d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q12.plan.txt @@ -2,20 +2,17 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q13.plan.txt index 30470310dca0..a4d7d7d75438 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q13.plan.txt @@ -8,23 +8,20 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q14.plan.txt index 21eb6df883d1..c8326b25e125 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) - partial aggregation over (i_item_sk_13) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) + partial aggregation over (i_item_sk_13) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) + partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) + partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) + partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_469", "i_category_id_473", "i_class_id_471"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_518) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) - partial aggregation over (i_item_sk_518) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_542, category_id_544, class_id_543) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) + scan item + final aggregation over (i_item_sk_518) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) + partial aggregation over (i_item_sk_518) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) + scan item + final aggregation over (brand_id_542, category_id_544, class_id_543) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) + partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) + partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) + partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -183,66 +171,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1052", "i_category_id_1056", "i_class_id_1054"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_1101) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) - partial aggregation over (i_item_sk_1101) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_1125, category_id_1127, class_id_1126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) + scan item + final aggregation over (i_item_sk_1101) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) + partial aggregation over (i_item_sk_1101) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) + scan item + final aggregation over (brand_id_1125, category_id_1127, class_id_1126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) + partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) + partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) + partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q15.plan.txt index b22f9d12537c..56302f05c739 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q16.plan.txt index d45ed9289047..5b6df301073a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q16.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (cr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_23", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_23", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q17.plan.txt index 7a9af86c0c6c..49c98c0b8dda 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q17.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_state) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q18.plan.txt index 7f9cbb3ebf63..5f2dfaaeb4fa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q18.plan.txt @@ -4,35 +4,33 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) remote exchange (REPARTITION, HASH, ["ca_country$gid", "ca_county$gid", "ca_state$gid", "groupid", "i_item_id$gid"]) partial aggregation over (ca_country$gid, ca_county$gid, ca_state$gid, groupid, i_item_id$gid) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) + remote exchange (REPARTITION, HASH, ["ca_country", "i_item_id"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) + scan customer_demographics local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q19.plan.txt index 967aa874de69..c500acf6fca3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q19.plan.txt @@ -7,26 +7,23 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q20.plan.txt index 2dce6eadbb48..852a9d2a3e5c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q20.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q21.plan.txt index e8067ff5be06..50e40917c430 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q23.plan.txt index 1ef6f718b353..c206b58e5ccb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_6, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_6, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_43) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_43) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_43) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_43"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_71) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_71"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_93"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_185) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_185"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_185) - final aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_185", "substr$gid_265"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_185) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_185"]) + partial aggregation over (ss_item_sk_185) + final aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_185", "substr$gid_265"]) + partial aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_274) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_274) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_274) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_274"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_322) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_322"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_344"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q24.plan.txt index f9922ec7376e..14c2547a1059 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q24.plan.txt @@ -5,40 +5,35 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,23 +52,19 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_13", "ss_ticket_number_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_38", "sr_ticket_number_45"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_113"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_142", "upper_159"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q25.plan.txt index 4dcfe955597d..8eedb8e05888 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q25.plan.txt @@ -7,35 +7,31 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q26.plan.txt index dc978ab9fbae..555bb67d3baa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q26.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q27.plan.txt index 22ec76d44200..b637b1d27f97 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q27.plan.txt @@ -4,23 +4,23 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, HASH, ["groupid", "i_item_id$gid", "s_state$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_item_id$gid", "s_state$gid"]) partial aggregation over (groupid, i_item_id$gid, s_state$gid) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, HASH, ["i_item_id"]) + remote exchange (REPARTITION, HASH, ["i_item_id"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q28.plan.txt index fcb9b4b7dd71..8f8331d2d85d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_125"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q29.plan.txt index 4dcfe955597d..8eedb8e05888 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q29.plan.txt @@ -7,35 +7,31 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q30.plan.txt index 740f81a2504b..ba844c8c4e25 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_85) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_85"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_85) - final aggregation over (ca_state_85, wr_returning_customer_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_28"]) - partial aggregation over (ca_state_85, wr_returning_customer_sk_28) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ca_state_85) + final aggregation over (ca_state_85, wr_returning_customer_sk_28) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_28"]) + partial aggregation over (ca_state_85, wr_returning_customer_sk_28) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_31"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q31.plan.txt index 16b1e2940a8f..2eea355e464a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_66, d_qoy_39, d_year_35) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_140, d_qoy_113, d_year_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) - partial aggregation over (ca_county_140, d_qoy_113, d_year_109) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_140, d_qoy_113, d_year_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) + partial aggregation over (ca_county_140, d_qoy_113, d_year_109) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_84"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_276, d_qoy_249, d_year_245) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_269"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_295"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_191, d_qoy_164, d_year_160) + final aggregation over (ca_county_361, d_qoy_334, d_year_330) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) - partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) + partial aggregation over (ca_county_361, d_qoy_334, d_year_330) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_295"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_191, d_qoy_164, d_year_160) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) + partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q32.plan.txt index 49bb921af9d5..d2a7e4049c81 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_19) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_19) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q33.plan.txt index 2502d90464f0..1b2427d4e654 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q33.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_6) + partial aggregation over (i_manufact_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_22) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_22) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) + partial aggregation over (i_manufact_id_22) + scan item partial aggregation over (i_manufact_id_95) final aggregation over (i_manufact_id_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_95) + partial aggregation over (i_manufact_id_95) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_121) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_121) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_121) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) + partial aggregation over (i_manufact_id_121) + scan item partial aggregation over (i_manufact_id_196) final aggregation over (i_manufact_id_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_196) + partial aggregation over (i_manufact_id_196) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_222) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_222) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_222) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) + partial aggregation over (i_manufact_id_222) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q34.plan.txt index 861d0f0bd0d8..8b6f823e1fe3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q34.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q35.plan.txt index 8353da6955f2..f94ec8be8411 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q35.plan.txt @@ -11,45 +11,39 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q36.plan.txt index c7aa800fe3c9..d640f5ac0d14 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q36.plan.txt @@ -5,18 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q37.plan.txt index 1cad16277680..2582d9fa4122 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q38.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q39.plan.txt index 0d241239eb7b..657fe01edba1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_9", "inv_warehouse_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_9", "inv_warehouse_sk_10", "w_warehouse_name_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + final aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_9", "inv_warehouse_sk_10", "w_warehouse_name_40"]) + partial aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q40.plan.txt index 62288a2c3dc5..266142fc4c1a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q42.plan.txt index d6a3c8cac82b..17cc12f10e9e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q44.plan.txt index e553c8e97eed..d84084843142 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_7) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) + partial aggregation over (ss_store_sk_7) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_80"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_28) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_28"]) + partial aggregation over (ss_item_sk_28) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_7) + final aggregation over (ss_store_sk_60) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_7) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_28"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_28) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_60) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_60"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_60) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_60"]) + partial aggregation over (ss_store_sk_60) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q45.plan.txt index b275b76606e4..5b30cdf7426c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q45.plan.txt @@ -10,23 +10,20 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q46.plan.txt index 6ddeddd43120..e8aa3db7886c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q46.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q47.plan.txt index 93faff459e9c..2d63bde52f55 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) + partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) + partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q48.plan.txt index 4e0866c981c0..7eb552bd09e3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q48.plan.txt @@ -7,20 +7,17 @@ final aggregation over () remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q49.plan.txt index 66381314bec4..6f687392953d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q49.plan.txt @@ -3,65 +3,56 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_17", "expr_21", "rank", "rank_19", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - partial aggregation over (wr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - partial aggregation over (cr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - partial aggregation over (sr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q50.plan.txt index c28e1f11bc9a..c2a9d481bf10 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q50.plan.txt @@ -8,16 +8,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q51.plan.txt index 06b2887fa30d..ed5e8607b930 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_7, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_7, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_7, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) + partial aggregation over (d_date_7, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q52.plan.txt index ab4fb497ba27..33752e693e6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q53.plan.txt index 2206a00753f9..cd10ab3698e6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q53.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy, i_manufact_id) + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q54.plan.txt index 73276be84ef1..23ee4dacf66b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_104) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_104) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_104) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_104"]) + partial aggregation over (expr_104) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q55.plan.txt index ad84f78b05f8..0322599f01e5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q56.plan.txt index b5c953a4d97e..e96c8c31aabb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q56.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q57.plan.txt index 25bb57998645..26f16c8477dc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) + partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) + partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q58.plan.txt index 7bd45a0070f2..192c6d359edd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q58.plan.txt @@ -7,31 +7,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_69) local exchange (GATHER, SINGLE, []) @@ -39,59 +36,53 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_69) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_126"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_126) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_126"]) + partial aggregation over (d_date_126) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_191) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_191"]) partial aggregation over (i_item_id_191) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_248) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_248"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_248) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_248) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_248"]) + partial aggregation over (d_date_248) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_190"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q59.plan.txt index 72c157ff0704..ae30fe8f578e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q59.plan.txt @@ -4,53 +4,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_53) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_75, ss_store_sk_53) - final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_75, ss_store_sk_53) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_53"]) + partial aggregation over (d_week_seq_75, ss_store_sk_53) + final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_53"]) + partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_sk_117"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q60.plan.txt index 70f98c7eb54e..4616b0ad95b6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q60.plan.txt @@ -6,83 +6,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_6"]) + scan item + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_83"]) + scan item + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_id_184"]) + scan item + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q61.plan.txt index 2aa259b1de4f..f5183d67d01e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q61.plan.txt @@ -6,28 +6,26 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan promotion @@ -37,25 +35,23 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q62.plan.txt index c66895aac6b8..90f04a71a65b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan ship_mode diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q63.plan.txt index 1732f85548b8..23e28eddc4b7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q63.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_manager_id) + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q64.plan.txt index 1d630ce9b299..b3b3aa65cb46 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q64.plan.txt @@ -3,94 +3,166 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_134"]) + final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_134"]) + partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_136"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_135"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk_134", "ss_ticket_number_141"]) join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk_159", "sr_ticket_number_166"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk_194) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_194"]) + partial aggregation over (cs_item_sk_194) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_194", "cs_order_number_196"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_217", "cr_order_number_231"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -98,100 +170,10 @@ remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_134"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_134"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_cdemo_sk_136"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_135"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_134", "ss_ticket_number_141"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_159", "sr_ticket_number_166"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_194) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_194"]) - partial aggregation over (cs_item_sk_194) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_194", "cs_order_number_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_217", "cr_order_number_231"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q65.plan.txt index 5d8a33cc5245..d464032d91fc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_8, ss_store_sk_13) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_8", "ss_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_8, ss_store_sk_13) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_8, ss_store_sk_13) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q66.plan.txt index aa228d59f437..b87ae78025f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_372, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) final aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) + partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q68.plan.txt index 52aa91be911d..5feab810ee06 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q68.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q69.plan.txt index 8de8d86fbc94..4b87cab3294e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q69.plan.txt @@ -7,47 +7,42 @@ local exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q70.plan.txt index 542ab85591d7..0ba93430749a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_53) final aggregation over (s_state_53) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_53) + partial aggregation over (s_state_53) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q72.plan.txt index bdbdae78e9d9..9c47f45ae5a1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q72.plan.txt @@ -9,37 +9,34 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -48,5 +45,4 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q73.plan.txt index 9fa35d4ee8c7..30d700ed0193 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q73.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q74.plan.txt index 23b39e29816e..3000b513f3ec 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q74.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) + partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_91"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) + final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) - partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) + partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_91"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_352"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) - partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_352"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) + final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) + partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_531"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) - partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_531"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q75.plan.txt index 7e7a9117d8eb..fef6864e7254 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145) final aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145, sales_amt_147, sales_cnt_146) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_198", "i_category_id_202", "i_class_id_200", "i_manufact_id_204"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_247", "cr_order_number_261"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_170", "cs_order_number_172"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_247", "cr_order_number_261"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_170", "cs_order_number_172"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_357", "sr_ticket_number_364"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_278", "ss_ticket_number_285"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_357", "sr_ticket_number_364"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_278", "ss_ticket_number_285"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_471", "wr_order_number_482"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_382", "ws_order_number_396"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_471", "wr_order_number_482"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_382", "ws_order_number_396"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q76.plan.txt index c1d7f408a238..96f6228260dc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q76.plan.txt @@ -3,44 +3,38 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_10", "d_year_9", "expr_134", "expr_135", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item remote exchange (REPARTITION, HASH, ["d_qoy_51", "d_year_47", "expr_140", "expr_141", "i_category_29"]) partial aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_111", "d_year_107", "expr_131", "expr_133", "i_category_89"]) partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q78.plan.txt index 4f6eec106eaa..50161185b413 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q78.plan.txt @@ -2,56 +2,48 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_year, ss_customer_sk, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + join (INNER, PARTITIONED): + final aggregation over (d_year, ss_customer_sk, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) + partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) - partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) + partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) + partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q81.plan.txt index f0c2e9ce4e42..03e019fc0d5f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q81.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) - partial aggregation over (ca_state, cr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_state, cr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cr_returning_customer_sk"]) + partial aggregation over (ca_state, cr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_88) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_88"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_88) - final aggregation over (ca_state_88, cr_returning_customer_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_28"]) - partial aggregation over (ca_state_88, cr_returning_customer_sk_28) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_88) + final aggregation over (ca_state_88, cr_returning_customer_sk_28) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_28"]) + partial aggregation over (ca_state_88, cr_returning_customer_sk_28) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_31"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q82.plan.txt index 4232c710d2d6..532952036fc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q83.plan.txt index 6023a24c3a48..d5992a1244c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q83.plan.txt @@ -7,34 +7,30 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_40) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) + partial aggregation over (d_week_seq_40) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -42,65 +38,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_131) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_131"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_131) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_165) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_165) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_131) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_131"]) + partial aggregation over (d_date_131) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_165) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) + partial aggregation over (d_week_seq_165) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_201) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_201"]) partial aggregation over (i_item_id_201) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_258) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_258"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_258) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_292) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_292) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_258) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_258"]) + partial aggregation over (d_date_258) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_292) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) + partial aggregation over (d_week_seq_292) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q85.plan.txt index b5198e5c38f0..2a72b1dbbed5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q85.plan.txt @@ -3,41 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk_6", "cd_education_status_9", "cd_marital_status_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_education_status", "cd_marital_status", "wr_returning_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk_6", "cd_education_status_9", "cd_marital_status_8"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_education_status", "cd_marital_status", "wr_returning_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_refunded_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q86.plan.txt index a3b94e200aa6..cca927926827 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q87.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q89.plan.txt index b5495ae3801b..396d705bbc38 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q91.plan.txt index 2da8cda3eecd..750e4be2dcb9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q91.plan.txt @@ -8,32 +8,28 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q92.plan.txt index 03866eaa7ffd..55eb5a4a778f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_7) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_7) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q94.plan.txt index cf882450c774..821a5b58e819 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q94.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_23", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_23", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) + partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q95.plan.txt index 21e3f3d023f2..aaf3edd0ed42 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q95.plan.txt @@ -9,41 +9,35 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_137"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales join (INNER, PARTITIONED): final aggregation over (ws_order_number_23) local exchange (GATHER, SINGLE, []) partial aggregation over (ws_order_number_23) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q98.plan.txt index e5840c437809..9775b6c511f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q98.plan.txt @@ -3,18 +3,16 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q99.plan.txt index 652884fb1181..7f8838ac985a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/orc/unpartitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q01.plan.txt index 630021f6984d..d091b5f71d8b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q01.plan.txt @@ -4,42 +4,37 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (sr_store_sk_12) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_12) - final aggregation over (sr_customer_sk_8, sr_store_sk_12) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_8", "sr_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_8, sr_store_sk_12) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_12) + final aggregation over (sr_customer_sk_8, sr_store_sk_12) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_8", "sr_store_sk_12"]) + partial aggregation over (sr_customer_sk_8, sr_store_sk_12) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q02.plan.txt index a9e4ac5bcfad..f26bc3efa106 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_124) - final aggregation over (d_day_name_134, d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) - partial aggregation over (d_day_name_134, d_week_seq_124) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_124) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) + partial aggregation over (d_week_seq_124) + final aggregation over (d_day_name_134, d_week_seq_124) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) + partial aggregation over (d_day_name_134, d_week_seq_124) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q04.plan.txt index 954de084f516..84c91be8eac4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q04.plan.txt @@ -4,117 +4,99 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) + partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_869"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) + final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) - partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) + partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_869"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1567"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) - partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1567"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) - partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_569"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) - partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1267"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) - partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) + partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_183"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_569"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) + final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) + partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1267"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) + final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) + partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_183"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q05.plan.txt index da6352ecc72f..169f66124bda 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_81", "ws_order_number_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q06.plan.txt index b27486cb2c12..f25b809ee8ef 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q06.plan.txt @@ -9,33 +9,29 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_11) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) + partial aggregation over (d_month_seq_11) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -44,9 +40,8 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_50) - scan item + partial aggregation over (i_category_50) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q07.plan.txt index d209bd09ae61..32f4763b443d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q07.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q08.plan.txt index 998b0b959fcb..45b29afc325b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_35"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_26) - scan customer_address + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_29) - final aggregation over (ca_zip_16) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_16"]) - partial aggregation over (ca_zip_16) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_29) + final aggregation over (ca_zip_16) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_16"]) + partial aggregation over (ca_zip_16) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q10.plan.txt index a53dd7834678..7d60d2a5fdd5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q10.plan.txt @@ -3,54 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q11.plan.txt index 6fee2f7f3db5..668a1795f1c6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q11.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) + partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_100"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) + final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) - partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) + partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_380"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) - partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_380"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) + final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) + partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_573"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) - partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_573"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q12.plan.txt index 14d23e717ece..6f94c98c5224 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q12.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q14.plan.txt index 21eb6df883d1..c8326b25e125 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) - partial aggregation over (i_item_sk_13) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) + partial aggregation over (i_item_sk_13) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) + partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) + partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) + partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_469", "i_category_id_473", "i_class_id_471"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_518) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) - partial aggregation over (i_item_sk_518) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_542, category_id_544, class_id_543) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) + scan item + final aggregation over (i_item_sk_518) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) + partial aggregation over (i_item_sk_518) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) + scan item + final aggregation over (brand_id_542, category_id_544, class_id_543) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) + partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) + partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) + partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -183,66 +171,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1052", "i_category_id_1056", "i_class_id_1054"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_1101) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) - partial aggregation over (i_item_sk_1101) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_1125, category_id_1127, class_id_1126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) + scan item + final aggregation over (i_item_sk_1101) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) + partial aggregation over (i_item_sk_1101) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) + scan item + final aggregation over (brand_id_1125, category_id_1127, class_id_1126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) + partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) + partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) + partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q15.plan.txt index 4a9d57b8e143..f359fbefb40f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q16.plan.txt index 4a4b7afa9faa..50f04961003b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q16.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (cr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_22", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_22", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_22, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q17.plan.txt index 326846b8f36d..47c73f484403 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q17.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_state) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q18.plan.txt index 9448c90ea1ed..3cdca5dac93e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q18.plan.txt @@ -17,16 +17,13 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan customer_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q19.plan.txt index 1862b92ad711..57e559af38d4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q19.plan.txt @@ -6,27 +6,24 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q20.plan.txt index 2dce6eadbb48..852a9d2a3e5c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q20.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q21.plan.txt index e8067ff5be06..50e40917c430 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q23.plan.txt index 64f6afa02051..a98a247c3340 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_6, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_6, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_42) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_42) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_42) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_70) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_70"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_93"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_184) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_184) - final aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_184", "substr$gid_265"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_184) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_184"]) + partial aggregation over (ss_item_sk_184) + final aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_184", "substr$gid_265"]) + partial aggregation over (d_date_210, ss_item_sk_184, substr$gid_265) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_273) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_273) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_273) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_273"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_321) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_321"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_344"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q24.plan.txt index 202a04950073..d4aee5a72c72 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q24.plan.txt @@ -5,39 +5,34 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -55,24 +50,20 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk_13"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_12", "ss_ticket_number_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_37", "sr_ticket_number_44"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_113"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_142", "upper_159"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q25.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q25.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q26.plan.txt index dc978ab9fbae..555bb67d3baa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q26.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q27.plan.txt index 22ec76d44200..ed0e2e73e9f3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q27.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_item_id$gid, s_state$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q28.plan.txt index f64d5530bbf2..069ba6e93218 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q29.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q29.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q30.plan.txt index 87c5b1a7671d..b2f63c78e14c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_85) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_85"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_85) - final aggregation over (ca_state_85, wr_returning_customer_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_27"]) - partial aggregation over (ca_state_85, wr_returning_customer_sk_27) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_30"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_85) + final aggregation over (ca_state_85, wr_returning_customer_sk_27) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_27"]) + partial aggregation over (ca_state_85, wr_returning_customer_sk_27) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_30"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q31.plan.txt index c65ae2ca3c5a..04a7a0fc2cd4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_66, d_qoy_39, d_year_35) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_140, d_qoy_113, d_year_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) - partial aggregation over (ca_county_140, d_qoy_113, d_year_109) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_140, d_qoy_113, d_year_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) + partial aggregation over (ca_county_140, d_qoy_113, d_year_109) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_83"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_276, d_qoy_249, d_year_245) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_269"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_294"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_191, d_qoy_164, d_year_160) + final aggregation over (ca_county_361, d_qoy_334, d_year_330) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) - partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) + partial aggregation over (ca_county_361, d_qoy_334, d_year_330) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_294"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_191, d_qoy_164, d_year_160) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) + partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q32.plan.txt index be2a74f9a07a..e09ec6de273c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_18) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_18) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_18) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q33.plan.txt index 2502d90464f0..1b2427d4e654 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q33.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_6) + partial aggregation over (i_manufact_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_22) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_22) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) + partial aggregation over (i_manufact_id_22) + scan item partial aggregation over (i_manufact_id_95) final aggregation over (i_manufact_id_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_95) + partial aggregation over (i_manufact_id_95) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_121) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_121) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_121) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) + partial aggregation over (i_manufact_id_121) + scan item partial aggregation over (i_manufact_id_196) final aggregation over (i_manufact_id_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_196) + partial aggregation over (i_manufact_id_196) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_222) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_222) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_222) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) + partial aggregation over (i_manufact_id_222) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q34.plan.txt index 861d0f0bd0d8..8b6f823e1fe3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q34.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q35.plan.txt index b7f586b43c2d..f1fed4317214 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q35.plan.txt @@ -13,43 +13,37 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q36.plan.txt index c7aa800fe3c9..d640f5ac0d14 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q36.plan.txt @@ -5,18 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q37.plan.txt index 1cad16277680..2582d9fa4122 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q38.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q39.plan.txt index b74a3c77c64a..eb2349ef1574 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_8", "inv_warehouse_sk_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_8", "inv_warehouse_sk_9", "w_warehouse_name_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + final aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_8", "inv_warehouse_sk_9", "w_warehouse_name_40"]) + partial aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q40.plan.txt index 62288a2c3dc5..266142fc4c1a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q42.plan.txt index aaf625b30700..87792b40cd98 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q44.plan.txt index f8aeb77ec755..7cd4ca5a0807 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) + partial aggregation over (ss_store_sk_6) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_80"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_27) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_27"]) + partial aggregation over (ss_item_sk_27) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_6) + final aggregation over (ss_store_sk_59) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_6) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_27"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_27) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_59) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_59) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_59"]) + partial aggregation over (ss_store_sk_59) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q45.plan.txt index 6c9c571af0b5..6a42b74e28db 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q45.plan.txt @@ -7,26 +7,23 @@ local exchange (GATHER, SINGLE, []) semijoin (REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q46.plan.txt index 237c49065a38..d675e12143ee 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q46.plan.txt @@ -2,37 +2,32 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q47.plan.txt index 93faff459e9c..2d63bde52f55 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) + partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) + partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q49.plan.txt index 66381314bec4..6f687392953d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q49.plan.txt @@ -3,65 +3,56 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_17", "expr_21", "rank", "rank_19", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - partial aggregation over (wr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - partial aggregation over (cr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - partial aggregation over (sr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q50.plan.txt index c28e1f11bc9a..c2a9d481bf10 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q50.plan.txt @@ -8,16 +8,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q51.plan.txt index 06b2887fa30d..ed5e8607b930 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_7, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_7, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_7, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) + partial aggregation over (d_date_7, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q52.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q53.plan.txt index 2206a00753f9..cd10ab3698e6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q53.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy, i_manufact_id) + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q54.plan.txt index 73276be84ef1..23ee4dacf66b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_104) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_104) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_104) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_104"]) + partial aggregation over (expr_104) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q55.plan.txt index 92c0bba58211..f89f2f078b63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q56.plan.txt index 926ac0f3580d..35ea332fb456 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q56.plan.txt @@ -6,30 +6,28 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) @@ -37,29 +35,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_83) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_67"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) @@ -67,26 +62,23 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_184) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_168"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q57.plan.txt index 7934a5c0f90a..105ad5f39b4a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) + partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) + partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q58.plan.txt index 1095ccb2b692..00602f5115c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q58.plan.txt @@ -4,90 +4,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + join (INNER, PARTITIONED): + final aggregation over (i_item_id_69) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_69"]) + partial aggregation over (i_item_id_69) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) + final aggregation over (d_date_126) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_126"]) + partial aggregation over (d_date_126) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_69) + final aggregation over (i_item_id_191) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_69) - join (INNER, REPLICATED): + remote exchange (REPARTITION, HASH, ["i_item_id_191"]) + partial aggregation over (i_item_id_191) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, REPLICATED): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_126) + final aggregation over (d_date_248) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_126"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_126) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - final aggregation over (i_item_id_191) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_191"]) - partial aggregation over (i_item_id_191) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_248) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_248"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_248) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_248"]) + partial aggregation over (d_date_248) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_190"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q59.plan.txt index 133dd26321d0..7f584558007b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q59.plan.txt @@ -4,53 +4,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_52) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_75, ss_store_sk_52) - final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_52"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_75, ss_store_sk_52) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_52"]) + partial aggregation over (d_week_seq_75, ss_store_sk_52) + final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_52"]) + partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_sk_117"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q60.plan.txt index b5c953a4d97e..e96c8c31aabb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q60.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q61.plan.txt index 9205d0d586d7..ff3d350a643f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q61.plan.txt @@ -6,28 +6,26 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan promotion @@ -37,25 +35,23 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q62.plan.txt index a00a28581512..61172c74fc4f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q63.plan.txt index 1732f85548b8..23e28eddc4b7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q63.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_manager_id) + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q64.plan.txt index 35e8a006c3c1..34baf2cecca6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q64.plan.txt @@ -3,195 +3,177 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_99"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_99"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_133"]) - partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_461"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_133"]) + partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_133) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_461"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk_373"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk_373"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_134"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_134"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_133", "ss_ticket_number_140"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk_133", "ss_ticket_number_140"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_158", "sr_ticket_number_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_193) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_193"]) - partial aggregation over (cs_item_sk_193) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_193", "cs_order_number_195"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_216", "cr_order_number_230"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk_158", "sr_ticket_number_165"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk_193) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk_193"]) + partial aggregation over (cs_item_sk_193) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_193", "cs_order_number_195"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_216", "cr_order_number_230"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q65.plan.txt index 16047fe4c95e..0c80486a2845 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_7, ss_store_sk_12) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_7", "ss_store_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_7, ss_store_sk_12) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_7, ss_store_sk_12) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q66.plan.txt index aa228d59f437..b87ae78025f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_372, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) final aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) + partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q68.plan.txt index e4e44d8682f3..0df5521a9d32 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q68.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q69.plan.txt index bc516ac6a348..d304792469c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q69.plan.txt @@ -3,55 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (LEFT, PARTITIONED): - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - final aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (LEFT, PARTITIONED): + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q70.plan.txt index 542ab85591d7..0ba93430749a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_53) final aggregation over (s_state_53) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_53) + partial aggregation over (s_state_53) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q72.plan.txt index bdbdae78e9d9..9c47f45ae5a1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q72.plan.txt @@ -9,37 +9,34 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -48,5 +45,4 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q73.plan.txt index 9fa35d4ee8c7..30d700ed0193 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q73.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q74.plan.txt index d9e31e2a2fb7..1faae3d6738a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q74.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) + partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_90"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) + final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) - partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) + partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_351"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) - partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_351"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) + final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) + partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_530"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) - partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_530"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q75.plan.txt index cb8fd860379c..72ac1ec6cf65 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145) final aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145, sales_amt_147, sales_cnt_146) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_198", "i_category_id_202", "i_class_id_200", "i_manufact_id_204"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_246", "cr_order_number_260"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_169", "cs_order_number_171"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_246", "cr_order_number_260"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_169", "cs_order_number_171"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_356", "sr_ticket_number_363"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_277", "ss_ticket_number_284"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_356", "sr_ticket_number_363"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_277", "ss_ticket_number_284"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_470", "wr_order_number_481"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_381", "ws_order_number_395"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_470", "wr_order_number_481"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_381", "ws_order_number_395"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q76.plan.txt index cadc9a68bf31..044c8310d542 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q76.plan.txt @@ -3,42 +3,37 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_10", "d_year_9", "expr_134", "expr_135", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_51", "d_year_47", "expr_140", "expr_141", "i_category_29"]) partial aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_111", "d_year_107", "expr_131", "expr_133", "i_category_89"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) + partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q78.plan.txt index 4f6eec106eaa..50161185b413 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q78.plan.txt @@ -2,56 +2,48 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_year, ss_customer_sk, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + join (INNER, PARTITIONED): + final aggregation over (d_year, ss_customer_sk, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) + partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) - partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) + partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) + partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q81.plan.txt index fa16645195a4..88dfbd2615ea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q81.plan.txt @@ -9,16 +9,14 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state, cr_returning_customer_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -31,24 +29,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state_88) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_88"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_88) - final aggregation over (ca_state_88, cr_returning_customer_sk_27) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_27"]) - partial aggregation over (ca_state_88, cr_returning_customer_sk_27) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_30"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_88) + final aggregation over (ca_state_88, cr_returning_customer_sk_27) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_27"]) + partial aggregation over (ca_state_88, cr_returning_customer_sk_27) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_30"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q82.plan.txt index 4232c710d2d6..532952036fc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q83.plan.txt index 75ea801d2649..108710add4fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q83.plan.txt @@ -7,34 +7,30 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_40) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) + partial aggregation over (d_week_seq_40) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -42,65 +38,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_131) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_131"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_131) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_165) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_165) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_131) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_131"]) + partial aggregation over (d_date_131) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_165) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) + partial aggregation over (d_week_seq_165) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_201) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_201"]) partial aggregation over (i_item_id_201) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_258) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_258"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_258) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_292) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_292) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_258) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_258"]) + partial aggregation over (d_date_258) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_292) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) + partial aggregation over (d_week_seq_292) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q85.plan.txt index f5f3ea2e69f3..23eed8958c96 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q85.plan.txt @@ -3,39 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q86.plan.txt index a3b94e200aa6..cca927926827 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q87.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q89.plan.txt index b5495ae3801b..396d705bbc38 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q91.plan.txt index e22013240d22..2d679f5abc8e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q91.plan.txt @@ -4,33 +4,30 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_call_center_id", "cc_manager", "cc_name", "cd_education_status", "cd_marital_status"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) + partial aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q92.plan.txt index d4858936dcb2..2dfed78f9a62 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_6) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_6) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q94.plan.txt index 17e81dd8854b..48ebc0026b42 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q94.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_22", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_22", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) + partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_22, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q95.plan.txt index b5f1e9ba58e9..5e42003378f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q95.plan.txt @@ -9,41 +9,35 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_100"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_136"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales join (INNER, PARTITIONED): final aggregation over (ws_order_number_22) local exchange (GATHER, SINGLE, []) partial aggregation over (ws_order_number_22) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_58"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q98.plan.txt index e5840c437809..9775b6c511f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q98.plan.txt @@ -3,18 +3,16 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q99.plan.txt index ba34f85785e9..e7f593fe04e2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/partitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q01.plan.txt index 56f3c72ccdcc..114fb89fe851 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q01.plan.txt @@ -4,42 +4,37 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk, sr_store_sk) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + final aggregation over (sr_customer_sk, sr_store_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_store_sk"]) + partial aggregation over (sr_customer_sk, sr_store_sk) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (sr_store_sk_13) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk_13) - final aggregation over (sr_customer_sk_9, sr_store_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk_9", "sr_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_customer_sk_9, sr_store_sk_13) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (sr_store_sk_13) + final aggregation over (sr_customer_sk_9, sr_store_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk_9", "sr_store_sk_13"]) + partial aggregation over (sr_customer_sk_9, sr_store_sk_13) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q02.plan.txt index a9e4ac5bcfad..f26bc3efa106 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q02.plan.txt @@ -6,44 +6,39 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq) - final aggregation over (d_day_name, d_week_seq) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) - partial aggregation over (d_day_name, d_week_seq) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq) + final aggregation over (d_day_name, d_week_seq) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq"]) + partial aggregation over (d_day_name, d_week_seq) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_124) - final aggregation over (d_day_name_134, d_week_seq_124) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) - partial aggregation over (d_day_name_134, d_week_seq_124) - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + final aggregation over (d_week_seq_124) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_124"]) + partial aggregation over (d_week_seq_124) + final aggregation over (d_day_name_134, d_week_seq_124) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_134", "d_week_seq_124"]) + partial aggregation over (d_day_name_134, d_week_seq_124) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q03.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q03.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q04.plan.txt index 61aa19348540..56356d683fe6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q04.plan.txt @@ -4,117 +4,99 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + final aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) + partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_870"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) + final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_861", "c_customer_id_848", "c_email_address_863", "c_first_name_855", "c_last_name_856", "c_login_862", "c_preferred_cust_flag_857", "d_year_909"]) - partial aggregation over (c_birth_country_861, c_customer_id_848, c_email_address_863, c_first_name_855, c_last_name_856, c_login_862, c_preferred_cust_flag_857, d_year_909) + remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) + partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_870"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1568"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_847"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1545"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1558", "c_customer_id_1545", "c_email_address_1560", "c_first_name_1552", "c_last_name_1553", "c_login_1559", "c_preferred_cust_flag_1554", "d_year_1606"]) - partial aggregation over (c_birth_country_1558, c_customer_id_1545, c_email_address_1560, c_first_name_1552, c_last_name_1553, c_login_1559, c_preferred_cust_flag_1554, d_year_1606) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1568"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) - partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_570"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) - partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1268"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + final aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) - partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + remote exchange (REPARTITION, HASH, ["c_birth_country_561", "c_customer_id_548", "c_email_address_563", "c_first_name_555", "c_last_name_556", "c_login_562", "c_preferred_cust_flag_557", "d_year_609"]) + partial aggregation over (c_birth_country_561, c_customer_id_548, c_email_address_563, c_first_name_555, c_last_name_556, c_login_562, c_preferred_cust_flag_557, d_year_609) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_570"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_547"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_1245"]) + final aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_1258", "c_customer_id_1245", "c_email_address_1260", "c_first_name_1252", "c_last_name_1253", "c_login_1259", "c_preferred_cust_flag_1254", "d_year_1306"]) + partial aggregation over (c_birth_country_1258, c_customer_id_1245, c_email_address_1260, c_first_name_1252, c_last_name_1253, c_login_1259, c_preferred_cust_flag_1254, d_year_1306) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1268"]) join (INNER, REPLICATED): - scan store_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_1244"]) scan customer + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_customer_id_162"]) + final aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_175", "c_customer_id_162", "c_email_address_177", "c_first_name_169", "c_last_name_170", "c_login_176", "c_preferred_cust_flag_171", "d_year_212"]) + partial aggregation over (c_birth_country_175, c_customer_id_162, c_email_address_177, c_first_name_169, c_last_name_170, c_login_176, c_preferred_cust_flag_171, d_year_212) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_184"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q05.plan.txt index ded90e2ed237..a837004a655b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q05.plan.txt @@ -45,12 +45,10 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_82", "ws_order_number_96"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q06.plan.txt index b27486cb2c12..f25b809ee8ef 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q06.plan.txt @@ -9,33 +9,29 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (d_month_seq_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_month_seq_11) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (d_month_seq_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_month_seq_11"]) + partial aggregation over (d_month_seq_11) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -44,9 +40,8 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category_50) - scan item + partial aggregation over (i_category_50) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q07.plan.txt index d209bd09ae61..32f4763b443d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q07.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q08.plan.txt index 998b0b959fcb..45b29afc325b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q08.plan.txt @@ -6,36 +6,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["substr_35"]) final aggregation over (ca_zip) local exchange (REPARTITION, HASH, ["ca_zip"]) remote exchange (REPARTITION, HASH, ["ca_zip_26"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_26) - scan customer_address + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_zip_29) - final aggregation over (ca_zip_16) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_zip_16"]) - partial aggregation over (ca_zip_16) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (ca_zip_29) + final aggregation over (ca_zip_16) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_zip_16"]) + partial aggregation over (ca_zip_16) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_7"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q10.plan.txt index a53dd7834678..7d60d2a5fdd5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q10.plan.txt @@ -3,54 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_dep_college_count", "cd_dep_count", "cd_dep_employed_count", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (RIGHT, PARTITIONED): - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_dep_college_count, cd_dep_count, cd_dep_employed_count, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (RIGHT, PARTITIONED): + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q11.plan.txt index 9e37382a600e..b0c5b40468d0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q11.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + final aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) + partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_101"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) + final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_92", "c_customer_id_79", "c_email_address_94", "c_first_name_86", "c_last_name_87", "c_login_93", "c_preferred_cust_flag_88", "d_year_129"]) - partial aggregation over (c_birth_country_92, c_customer_id_79, c_email_address_94, c_first_name_86, c_last_name_87, c_login_93, c_preferred_cust_flag_88, d_year_129) + remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) + partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_381"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_78"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_358"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_371", "c_customer_id_358", "c_email_address_373", "c_first_name_365", "c_last_name_366", "c_login_372", "c_preferred_cust_flag_367", "d_year_419"]) - partial aggregation over (c_birth_country_371, c_customer_id_358, c_email_address_373, c_first_name_365, c_last_name_366, c_login_372, c_preferred_cust_flag_367, d_year_419) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_381"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) + partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) + final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "c_customer_id", "c_email_address", "c_first_name", "c_last_name", "c_login", "c_preferred_cust_flag", "d_year"]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) + partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_574"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_551"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country_564", "c_customer_id_551", "c_email_address_566", "c_first_name_558", "c_last_name_559", "c_login_565", "c_preferred_cust_flag_560", "d_year_612"]) - partial aggregation over (c_birth_country_564, c_customer_id_551, c_email_address_566, c_first_name_558, c_last_name_559, c_login_565, c_preferred_cust_flag_560, d_year_612) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_574"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_550"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q12.plan.txt index 14d23e717ece..6f94c98c5224 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q12.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q14.plan.txt index 21eb6df883d1..c8326b25e125 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q14.plan.txt @@ -9,66 +9,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_6", "i_category_id_8", "i_class_id_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) - partial aggregation over (i_item_sk_13) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id, category_id, class_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + scan item + final aggregation over (i_item_sk_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_13"]) + partial aggregation over (i_item_sk_13) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_20", "i_category_id_24", "i_class_id_22"]) + scan item + final aggregation over (brand_id, category_id, class_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_72", "i_category_id_76", "i_class_id_74"]) + partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_128", "i_category_id_132", "i_class_id_130"]) + partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_184", "i_category_id_188", "i_class_id_186"]) + partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -96,66 +90,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_469", "i_category_id_473", "i_class_id_471"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_518) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) - partial aggregation over (i_item_sk_518) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_542, category_id_544, class_id_543) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_462"]) + scan item + final aggregation over (i_item_sk_518) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_518"]) + partial aggregation over (i_item_sk_518) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_525", "i_category_id_529", "i_class_id_527"]) + scan item + final aggregation over (brand_id_542, category_id_544, class_id_543) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_580", "i_category_id_584", "i_class_id_582"]) + partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_670", "i_category_id_674", "i_class_id_672"]) + partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_760", "i_category_id_764", "i_class_id_762"]) + partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -183,66 +171,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_1052", "i_category_id_1056", "i_class_id_1054"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (i_item_sk_1101) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) - partial aggregation over (i_item_sk_1101) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item - final aggregation over (brand_id_1125, category_id_1127, class_id_1126) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["i_item_sk_1045"]) + scan item + final aggregation over (i_item_sk_1101) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_1101"]) + partial aggregation over (i_item_sk_1101) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["i_brand_id_1108", "i_category_id_1112", "i_class_id_1110"]) + scan item + final aggregation over (brand_id_1125, category_id_1127, class_id_1126) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_brand_id_1163", "i_category_id_1167", "i_class_id_1165"]) + partial aggregation over (i_brand_id_1163, i_category_id_1167, i_class_id_1165) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1253", "i_category_id_1257", "i_class_id_1255"]) + partial aggregation over (i_brand_id_1253, i_category_id_1257, i_class_id_1255) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + remote exchange (REPARTITION, HASH, ["i_brand_id_1343", "i_category_id_1347", "i_class_id_1345"]) + partial aggregation over (i_brand_id_1343, i_category_id_1347, i_class_id_1345) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q15.plan.txt index 4a9d57b8e143..f359fbefb40f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q15.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q16.plan.txt index 079fa5e01b79..93679bddc789 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q16.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (cr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_order_number) - scan catalog_returns + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_23", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "cc_county", "cs_call_center_sk", "cs_ext_ship_cost", "cs_net_profit", "cs_order_number_23", "cs_ship_addr_sk", "cs_ship_date_sk", "cs_warehouse_sk", "d_date", "unique"]) + partial aggregation over (ca_state, cc_county, cs_call_center_sk, cs_ext_ship_cost, cs_net_profit, cs_order_number_23, cs_ship_addr_sk, cs_ship_date_sk, cs_warehouse_sk, d_date, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q17.plan.txt index 326846b8f36d..47c73f484403 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q17.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_state) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q18.plan.txt index 9448c90ea1ed..3cdca5dac93e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q18.plan.txt @@ -17,16 +17,13 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk_2"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan customer_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q19.plan.txt index 1862b92ad711..57e559af38d4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q19.plan.txt @@ -6,27 +6,24 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand, i_brand_id, i_manufact, i_manufact_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q20.plan.txt index 2dce6eadbb48..852a9d2a3e5c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q20.plan.txt @@ -2,18 +2,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q21.plan.txt index e8067ff5be06..50e40917c430 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q21.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id, w_warehouse_name) + partial aggregation over (i_item_id, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q23.plan.txt index 1ef6f718b353..c206b58e5ccb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q23.plan.txt @@ -4,35 +4,31 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6, ss_item_sk, substr$gid) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + final aggregation over (d_date_6, ss_item_sk, substr$gid) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_6", "ss_item_sk", "substr$gid"]) + partial aggregation over (d_date_6, ss_item_sk, substr$gid) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_43) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_43) @@ -40,12 +36,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_43) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_43"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -57,48 +51,42 @@ final aggregation over () partial aggregation over (ss_customer_sk_71) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_71"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_93"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_185) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_185"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_185) - final aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_185", "substr$gid_265"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + join (INNER, PARTITIONED): + final aggregation over (ss_item_sk_185) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_185"]) + partial aggregation over (ss_item_sk_185) + final aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_210", "ss_item_sk_185", "substr$gid_265"]) + partial aggregation over (d_date_210, ss_item_sk_185, substr$gid_265) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (ss_customer_sk_274) cross join (can skip output duplicates): final aggregation over (ss_customer_sk_274) @@ -106,12 +94,10 @@ final aggregation over () partial aggregation over (ss_customer_sk_274) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_274"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -123,13 +109,11 @@ final aggregation over () partial aggregation over (ss_customer_sk_322) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_322"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_344"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q24.plan.txt index ac08a0209ca8..5126cd62beeb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q24.plan.txt @@ -5,39 +5,34 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (c_first_name, c_last_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_first_name, c_last_name, s_store_name) - final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) - partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (c_first_name, c_last_name, s_store_name) + final aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_first_name", "c_last_name", "ca_state", "i_color", "i_current_price", "i_manager_id", "i_size", "i_units", "s_state", "s_store_name"]) + partial aggregation over (c_first_name, c_last_name, ca_state, i_color, i_current_price, i_manager_id, i_size, i_units, s_state, s_store_name) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_zip", "upper"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_birth_country", "s_zip"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -55,24 +50,20 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk_14"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_13", "ss_ticket_number_20"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_38", "sr_ticket_number_45"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_113"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_zip_142", "upper_159"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q25.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q25.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q26.plan.txt index dc978ab9fbae..555bb67d3baa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q26.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q27.plan.txt index 22ec76d44200..ed0e2e73e9f3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q27.plan.txt @@ -6,21 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_item_id$gid, s_state$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q28.plan.txt index fcb9b4b7dd71..8f8331d2d85d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q28.plan.txt @@ -9,45 +9,39 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_97"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price_125"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q29.plan.txt index 8eeff963a4b4..a21872e9dd27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q29.plan.txt @@ -6,36 +6,32 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_desc, i_item_id, s_store_id, s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q30.plan.txt index 592dda966c28..c4f171f432f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q30.plan.txt @@ -4,54 +4,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, wr_returning_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) - partial aggregation over (ca_state, wr_returning_customer_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_state, wr_returning_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "wr_returning_customer_sk"]) + partial aggregation over (ca_state, wr_returning_customer_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (ca_state_85) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_85"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_85) - final aggregation over (ca_state_85, wr_returning_customer_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_28"]) - partial aggregation over (ca_state_85, wr_returning_customer_sk_28) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_85) + final aggregation over (ca_state_85, wr_returning_customer_sk_28) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_85", "wr_returning_customer_sk_28"]) + partial aggregation over (ca_state_85, wr_returning_customer_sk_28) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_returning_addr_sk_31"]) + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_77"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q31.plan.txt index 16b1e2940a8f..2eea355e464a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q31.plan.txt @@ -10,35 +10,30 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_66, d_qoy_39, d_year_35) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_140, d_qoy_113, d_year_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) - partial aggregation over (ca_county_140, d_qoy_113, d_year_109) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk_84"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + final aggregation over (ca_county_140, d_qoy_113, d_year_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_140", "d_qoy_113", "d_year_109"]) + partial aggregation over (ca_county_140, d_qoy_113, d_year_109) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk_84"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_133"]) + scan customer_address join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -46,68 +41,58 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (ca_county_276, d_qoy_249, d_year_245) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_269"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_295"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ca_county, d_qoy, d_year) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) - partial aggregation over (ca_county, d_qoy, d_year) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - final aggregation over (ca_county_191, d_qoy_164, d_year_160) + final aggregation over (ca_county_361, d_qoy_334, d_year_330) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) - partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + remote exchange (REPARTITION, HASH, ["ca_county_361", "d_qoy_334", "d_year_330"]) + partial aggregation over (ca_county_361, d_qoy_334, d_year_330) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_295"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + remote exchange (REPARTITION, HASH, ["ca_address_sk_354"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) + join (INNER, PARTITIONED): + final aggregation over (ca_county, d_qoy, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county", "d_qoy", "d_year"]) + partial aggregation over (ca_county, d_qoy, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + final aggregation over (ca_county_191, d_qoy_164, d_year_160) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_county_191", "d_qoy_164", "d_year_160"]) + partial aggregation over (ca_county_191, d_qoy_164, d_year_160) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_184"]) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q32.plan.txt index 49bb921af9d5..d2a7e4049c81 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q32.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (cs_item_sk_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_item_sk_19) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk_19) join (INNER, REPLICATED): scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q33.plan.txt index 2502d90464f0..1b2427d4e654 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q33.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_6) + partial aggregation over (i_manufact_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_22) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_22) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_22"]) + partial aggregation over (i_manufact_id_22) + scan item partial aggregation over (i_manufact_id_95) final aggregation over (i_manufact_id_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_95"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_95) + partial aggregation over (i_manufact_id_95) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_121) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_121) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_121) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_121"]) + partial aggregation over (i_manufact_id_121) + scan item partial aggregation over (i_manufact_id_196) final aggregation over (i_manufact_id_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_196) + partial aggregation over (i_manufact_id_196) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_manufact_id_222) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_manufact_id_222) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_manufact_id_222) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_manufact_id_222"]) + partial aggregation over (i_manufact_id_222) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q34.plan.txt index 861d0f0bd0d8..8b6f823e1fe3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q34.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q35.plan.txt index b7f586b43c2d..f1fed4317214 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q35.plan.txt @@ -13,43 +13,37 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address final aggregation over (ss_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + scan customer_demographics final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (cs_ship_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) + partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q36.plan.txt index c7aa800fe3c9..d640f5ac0d14 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q36.plan.txt @@ -5,18 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q37.plan.txt index 1cad16277680..2582d9fa4122 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q37.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q38.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q38.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q39.plan.txt index 0d241239eb7b..657fe01edba1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q39.plan.txt @@ -3,43 +3,39 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "inv_item_sk", "inv_warehouse_sk", "w_warehouse_name"]) + partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_9", "inv_warehouse_sk_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_9", "inv_warehouse_sk_10", "w_warehouse_name_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + final aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_62", "inv_item_sk_9", "inv_warehouse_sk_10", "w_warehouse_name_40"]) + partial aggregation over (d_moy_62, inv_item_sk_9, inv_warehouse_sk_10, w_warehouse_name_40) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q40.plan.txt index 62288a2c3dc5..266142fc4c1a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q40.plan.txt @@ -9,12 +9,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q41.plan.txt index c67211745712..58a96c8d1ec2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q41.plan.txt @@ -3,11 +3,10 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_product_name) - single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) - join (INNER, REPLICATED, can skip output duplicates): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + partial aggregation over (i_product_name) + single aggregation over (i_manufact, i_manufact_id, i_product_name, unique) + join (INNER, REPLICATED, can skip output duplicates): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q42.plan.txt index aaf625b30700..87792b40cd98 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q42.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_category", "i_category_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_category, i_category_id) + partial aggregation over (d_year, i_category, i_category_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q43.plan.txt index 53c0550a78af..9d9e36441a12 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q43.plan.txt @@ -3,19 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_store_id, s_store_name) - final aggregation over (d_day_name, s_store_id, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, s_store_id, s_store_name) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (s_store_id, s_store_name) + final aggregation over (d_day_name, s_store_id, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "s_store_id", "s_store_name"]) + partial aggregation over (d_day_name, s_store_id, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q44.plan.txt index e553c8e97eed..d84084843142 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q44.plan.txt @@ -4,48 +4,42 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + cross join: + final aggregation over (ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + partial aggregation over (ss_item_sk) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (ss_store_sk_7) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) + partial aggregation over (ss_store_sk_7) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["rank_80"]) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk) + final aggregation over (ss_item_sk_28) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk_28"]) + partial aggregation over (ss_item_sk_28) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_7) + final aggregation over (ss_store_sk_60) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_7) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["rank_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - cross join: - final aggregation over (ss_item_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_28"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_28) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (ss_store_sk_60) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_60"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk_60) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_store_sk_60"]) + partial aggregation over (ss_store_sk_60) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q45.plan.txt index 6c9c571af0b5..6a42b74e28db 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q45.plan.txt @@ -7,26 +7,23 @@ local exchange (GATHER, SINGLE, []) semijoin (REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q46.plan.txt index 237c49065a38..d675e12143ee 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q46.plan.txt @@ -2,37 +2,32 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q47.plan.txt index 93faff459e9c..2d63bde52f55 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q47.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "i_brand", "i_category", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_63", "d_year_61", "i_brand_14", "i_category_18", "s_company_name_102", "s_store_name_90"]) + partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + final aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_182", "d_year_180", "i_brand_133", "i_category_137", "s_company_name_221", "s_store_name_209"]) + partial aggregation over (d_moy_182, d_year_180, i_brand_133, i_category_137, s_company_name_221, s_store_name_209) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q49.plan.txt index 66381314bec4..6f687392953d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q49.plan.txt @@ -3,65 +3,56 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, currency_rank, item, return_rank, return_ratio) local exchange (REPARTITION, HASH, ["channel", "currency_rank", "item", "return_rank", "return_ratio"]) remote exchange (REPARTITION, HASH, ["expr_17", "expr_21", "rank", "rank_19", "wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (wr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - partial aggregation over (wr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_17, expr_21, rank, rank_19, wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (wr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + partial aggregation over (wr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (cr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - partial aggregation over (cr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_item_sk, expr_66, expr_70, rank_68, rank_69) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (cr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + partial aggregation over (cr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (sr_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - partial aggregation over (sr_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (expr_115, expr_120, rank_117, rank_118, sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (sr_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["sr_item_sk"]) + partial aggregation over (sr_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q50.plan.txt index c28e1f11bc9a..c2a9d481bf10 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q50.plan.txt @@ -8,16 +8,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk", "sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q51.plan.txt index 06b2887fa30d..ed5e8607b930 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q51.plan.txt @@ -2,31 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (FULL, PARTITIONED): - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date, ws_item_sk) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_date_7, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_7, ss_item_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (FULL, PARTITIONED): + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + final aggregation over (d_date, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date", "ws_item_sk"]) + partial aggregation over (d_date, ws_item_sk) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + final aggregation over (d_date_7, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_7", "ss_item_sk"]) + partial aggregation over (d_date_7, ss_item_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q52.plan.txt index 4728fd4dc83f..70aae3fa1bdd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q52.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, i_brand, i_brand_id) + partial aggregation over (d_year, i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q53.plan.txt index 2206a00753f9..cd10ab3698e6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q53.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_qoy, i_manufact_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy, i_manufact_id) + final aggregation over (d_qoy, i_manufact_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_qoy", "i_manufact_id"]) + partial aggregation over (d_qoy, i_manufact_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q54.plan.txt index 73276be84ef1..23ee4dacf66b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q54.plan.txt @@ -3,71 +3,64 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr$gid) - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - cross join: - cross join: - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + partial aggregation over (expr$gid) + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + cross join: + cross join: + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + join (INNER, REPLICATED): + scan customer_address local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_current_addr_sk, c_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (c_current_addr_sk, c_customer_sk) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + final aggregation over (c_current_addr_sk, c_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk", "c_customer_sk"]) + partial aggregation over (c_current_addr_sk, c_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - scan customer + remote exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - final aggregation over (expr_104) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (expr_104) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr"]) + partial aggregation over (expr) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + final aggregation over (expr_104) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["expr_104"]) + partial aggregation over (expr_104) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q55.plan.txt index 92c0bba58211..f89f2f078b63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q55.plan.txt @@ -3,14 +3,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_brand_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_brand, i_brand_id) + partial aggregation over (i_brand, i_brand_id) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q56.plan.txt index 926ac0f3580d..35ea332fb456 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q56.plan.txt @@ -6,30 +6,28 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) @@ -37,29 +35,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_83) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_67"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) @@ -67,26 +62,23 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_184) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_168"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q57.plan.txt index 7934a5c0f90a..105ad5f39b4a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q57.plan.txt @@ -4,64 +4,58 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name", "d_moy", "d_year", "i_brand", "i_category"]) + partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_102", "d_moy_74", "d_year_72", "i_brand_14", "i_category_18"]) + partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + final aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cc_name_232", "d_moy_204", "d_year_202", "i_brand_144", "i_category_148"]) + partial aggregation over (cc_name_232, d_moy_204, d_year_202, i_brand_144, i_category_148) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q58.plan.txt index 1095ccb2b692..00602f5115c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q58.plan.txt @@ -4,90 +4,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id) + partial aggregation over (i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + join (INNER, PARTITIONED): + final aggregation over (i_item_id_69) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_69"]) + partial aggregation over (i_item_id_69) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) + final aggregation over (d_date_126) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_126"]) + partial aggregation over (d_date_126) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_69) + final aggregation over (i_item_id_191) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_69) - join (INNER, REPLICATED): + remote exchange (REPARTITION, HASH, ["i_item_id_191"]) + partial aggregation over (i_item_id_191) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, REPLICATED): - scan catalog_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_126) + final aggregation over (d_date_248) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_126"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_126) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - final aggregation over (i_item_id_191) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_191"]) - partial aggregation over (i_item_id_191) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_248) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_248"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_248) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim + remote exchange (REPARTITION, HASH, ["d_date_248"]) + partial aggregation over (d_date_248) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_190"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q59.plan.txt index 72c157ff0704..ae30fe8f578e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q59.plan.txt @@ -4,53 +4,46 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq, ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq, ss_store_sk) - final aggregation over (d_day_name, d_week_seq, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name, d_week_seq, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_week_seq, ss_store_sk) + final aggregation over (d_day_name, d_week_seq, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name", "d_week_seq", "ss_store_sk"]) + partial aggregation over (d_day_name, d_week_seq, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_53) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_75, ss_store_sk_53) - final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + final aggregation over (d_week_seq_75, ss_store_sk_53) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["d_week_seq_75", "ss_store_sk_53"]) + partial aggregation over (d_week_seq_75, ss_store_sk_53) + final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_day_name_85", "d_week_seq_75", "ss_store_sk_53"]) + partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_53) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_sk_117"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q60.plan.txt index b5c953a4d97e..e96c8c31aabb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q60.plan.txt @@ -6,83 +6,77 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_6) + partial aggregation over (i_item_id_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_10) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_10) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_10) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_10"]) + partial aggregation over (i_item_id_10) + scan item partial aggregation over (i_item_id_83) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_83"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_83) + partial aggregation over (i_item_id_83) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_109) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_109"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_109) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_109) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_109"]) + partial aggregation over (i_item_id_109) + scan item partial aggregation over (i_item_id_184) final aggregation over (i_item_id_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_184"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_184) + partial aggregation over (i_item_id_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (i_item_id_210) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_210"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_item_id_210) - scan item + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (i_item_id_210) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_id_210"]) + partial aggregation over (i_item_id_210) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q61.plan.txt index 2aa259b1de4f..f5183d67d01e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q61.plan.txt @@ -6,28 +6,26 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan promotion @@ -37,25 +35,23 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_15"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q62.plan.txt index a00a28581512..61172c74fc4f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q62.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sm_type, substr$gid, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sm_type", "substr$gid", "web_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sm_type, substr$gid, web_name) + partial aggregation over (sm_type, substr$gid, web_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q63.plan.txt index 1732f85548b8..23e28eddc4b7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q63.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_manager_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_manager_id) + final aggregation over (d_moy, i_manager_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_manager_id"]) + partial aggregation over (d_moy, i_manager_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q64.plan.txt index bc2e8da4aafe..69658a2c0896 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q64.plan.txt @@ -3,195 +3,177 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) - partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_99"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city", "ca_city_105", "ca_street_name", "ca_street_name_102", "ca_street_number", "ca_street_number_101", "ca_zip", "ca_zip_108", "d_year", "d_year_15", "d_year_45", "i_product_name", "s_store_name", "s_zip", "ss_item_sk"]) + partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_99"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - partial aggregation over (cs_item_sk) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + partial aggregation over (cs_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_75"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_name_343", "s_zip_363", "ss_item_sk_134"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_134"]) - partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk_461"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_city_452", "ca_city_467", "ca_street_name_449", "ca_street_name_464", "ca_street_number_448", "ca_street_number_463", "ca_zip_455", "ca_zip_470", "d_year_254", "d_year_284", "d_year_314", "i_product_name_507", "s_store_name_343", "s_zip_363", "ss_item_sk_134"]) + partial aggregation over (ca_city_452, ca_city_467, ca_street_name_449, ca_street_name_464, ca_street_number_448, ca_street_number_463, ca_zip_455, ca_zip_470, d_year_254, d_year_284, d_year_314, i_product_name_507, s_store_name_343, s_zip_363, ss_item_sk_134) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk_461"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk_373"]) + join (INNER, REPLICATED): scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk_373"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_135"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk_371"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_135"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_134", "ss_ticket_number_141"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk_134", "ss_ticket_number_141"]) + join (INNER, REPLICATED): + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_159", "sr_ticket_number_166"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_194) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_194"]) - partial aggregation over (cs_item_sk_194) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk_194", "cs_order_number_196"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk_217", "cr_order_number_231"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + remote exchange (REPARTITION, HASH, ["sr_item_sk_159", "sr_ticket_number_166"]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer + final aggregation over (cs_item_sk_194) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + remote exchange (REPARTITION, HASH, ["cs_item_sk_194"]) + partial aggregation over (cs_item_sk_194) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk_194", "cs_order_number_196"]) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cr_item_sk_217", "cr_order_number_231"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_369"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cd_demo_sk_400"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan income_band + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + scan income_band diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q65.plan.txt index 5d8a33cc5245..d464032d91fc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q65.plan.txt @@ -5,34 +5,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_item_sk_8, ss_store_sk_13) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk_8", "ss_store_sk_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk_8, ss_store_sk_13) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_8, ss_store_sk_13) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) - final aggregation over (ss_item_sk, ss_store_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_item_sk, ss_store_sk) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk) + final aggregation over (ss_item_sk, ss_store_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_store_sk"]) + partial aggregation over (ss_item_sk, ss_store_sk) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q66.plan.txt index aa228d59f437..b87ae78025f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q66.plan.txt @@ -6,55 +6,51 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + partial aggregation over (d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "d_year", "w_city_2", "w_country_5", "w_county_3", "w_state_4", "w_warehouse_name_0", "w_warehouse_sq_ft_1"]) + partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse partial aggregation over (concat_372, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) final aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + partial aggregation over (d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + final aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy_86", "d_year_84", "w_city_70", "w_country_74", "w_county_71", "w_state_72", "w_warehouse_name_64", "w_warehouse_sq_ft_65"]) + partial aggregation over (d_moy_86, d_year_84, w_city_70, w_country_74, w_county_71, w_state_72, w_warehouse_name_64, w_warehouse_sq_ft_65) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan time_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q68.plan.txt index e4e44d8682f3..0df5521a9d32 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q68.plan.txt @@ -4,35 +4,30 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q69.plan.txt index bc516ac6a348..d304792469c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q69.plan.txt @@ -3,55 +3,48 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_credit_rating", "cd_education_status", "cd_gender", "cd_marital_status", "cd_purchase_estimate"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) - join (LEFT, PARTITIONED): - join (RIGHT, PARTITIONED): - final aggregation over (ws_bill_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_bill_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (ss_customer_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - final aggregation over (cs_ship_customer_sk) + partial aggregation over (cd_credit_rating, cd_education_status, cd_gender, cd_marital_status, cd_purchase_estimate) + join (LEFT, PARTITIONED): + join (RIGHT, PARTITIONED): + final aggregation over (ws_bill_customer_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_ship_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) + partial aggregation over (ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + join (INNER, PARTITIONED): + final aggregation over (ss_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + partial aggregation over (ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_address + final aggregation over (cs_ship_customer_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_ship_customer_sk"]) + partial aggregation over (cs_ship_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q70.plan.txt index 542ab85591d7..0ba93430749a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q70.plan.txt @@ -8,28 +8,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store single aggregation over (s_state_53) final aggregation over (s_state_53) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_state_53"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_state_53) + partial aggregation over (s_state_53) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q72.plan.txt index bdbdae78e9d9..9c47f45ae5a1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q72.plan.txt @@ -9,37 +9,34 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -48,5 +45,4 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q73.plan.txt index 9fa35d4ee8c7..30d700ed0193 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q73.plan.txt @@ -3,26 +3,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_ticket_number) + final aggregation over (ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q74.plan.txt index 23b39e29816e..3000b513f3ec 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q74.plan.txt @@ -3,78 +3,66 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + final aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) + partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk_91"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) + final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_69", "c_first_name_76", "c_last_name_77", "d_year_119"]) - partial aggregation over (c_customer_id_69, c_first_name_76, c_last_name_77, d_year_119) + remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) + partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk_91"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_352"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_68"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_329", "c_first_name_336", "c_last_name_337", "d_year_390"]) - partial aggregation over (c_customer_id_329, c_first_name_336, c_last_name_337, d_year_390) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_352"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) + scan customer local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) + partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) + final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id", "c_first_name", "c_last_name", "d_year"]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) + partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_531"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_id_508", "c_first_name_515", "c_last_name_516", "d_year_569"]) - partial aggregation over (c_customer_id_508, c_first_name_515, c_last_name_516, d_year_569) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_531"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_507"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q75.plan.txt index 7e7a9117d8eb..fef6864e7254 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q75.plan.txt @@ -5,119 +5,101 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand_id, i_category_id, i_class_id, i_manufact_id, sales_amt, sales_cnt) local exchange (REPARTITION, HASH, ["d_year", "i_brand_id", "i_category_id", "i_class_id", "i_manufact_id"]) remote exchange (REPARTITION, HASH, ["i_brand_id_7", "i_category_id_9", "i_class_id_8", "i_manufact_id_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_13, expr, expr_18, i_brand_id_7, i_category_id_9, i_class_id_8, i_manufact_id_10) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_51, expr_77, expr_78, i_brand_id_28, i_category_id_32, i_class_id_30, i_manufact_id_34) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_111, expr_137, expr_138, i_brand_id_88, i_category_id_92, i_class_id_90, i_manufact_id_94) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim single aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145) final aggregation over (d_year_141, i_brand_id_142, i_category_id_144, i_class_id_143, i_manufact_id_145, sales_amt_147, sales_cnt_146) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand_id_198", "i_category_id_202", "i_class_id_200", "i_manufact_id_204"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["cr_item_sk_247", "cr_order_number_261"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_item_sk_170", "cs_order_number_172"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_221, expr_274, expr_275, i_brand_id_198, i_category_id_202, i_class_id_200, i_manufact_id_204) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["cr_item_sk_247", "cr_order_number_261"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_item_sk_170", "cs_order_number_172"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["sr_item_sk_357", "sr_ticket_number_364"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_278", "ss_ticket_number_285"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_331, expr_377, expr_378, i_brand_id_308, i_category_id_312, i_class_id_310, i_manufact_id_314) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["sr_item_sk_357", "sr_ticket_number_364"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_item_sk_278", "ss_ticket_number_285"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) - join (RIGHT, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["wr_item_sk_471", "wr_order_number_482"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk_382", "ws_order_number_396"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_445, expr_495, expr_496, i_brand_id_422, i_category_id_426, i_class_id_424, i_manufact_id_428) + join (RIGHT, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["wr_item_sk_471", "wr_order_number_482"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk_382", "ws_order_number_396"]) + join (INNER, REPLICATED, can skip output duplicates): + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q76.plan.txt index cadc9a68bf31..044c8310d542 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q76.plan.txt @@ -3,42 +3,37 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, ["channel", "col_name", "d_qoy", "d_year", "i_category"]) remote exchange (REPARTITION, HASH, ["d_qoy_10", "d_year_9", "expr_134", "expr_135", "i_category_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_51", "d_year_47", "expr_140", "expr_141", "i_category_29"]) partial aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan date_dim + scan date_dim remote exchange (REPARTITION, HASH, ["d_qoy_111", "d_year_107", "expr_131", "expr_133", "i_category_89"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) + partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q77.plan.txt index a5c406789d66..f5f8b7b62eea 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q77.plan.txt @@ -9,80 +9,74 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_store_sk) + partial aggregation over (ss_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (sr_store_sk) + partial aggregation over (sr_store_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_call_center_sk) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk) + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over (cr_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_call_center_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_call_center_sk) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_web_page_sk) + partial aggregation over (ws_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_web_page_sk) + partial aggregation over (wr_web_page_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q78.plan.txt index 4f6eec106eaa..50161185b413 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q78.plan.txt @@ -2,56 +2,48 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, PARTITIONED): - final aggregation over (d_year, ss_customer_sk, ss_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + join (INNER, PARTITIONED): + final aggregation over (d_year, ss_customer_sk, ss_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year", "ss_customer_sk", "ss_item_sk"]) + partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) + scan store_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) - partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + final aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_year_17", "ws_bill_customer_sk", "ws_item_sk"]) + partial aggregation over (d_year_17, ws_bill_customer_sk, ws_item_sk) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + scan web_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) - join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk", "d_year_56"]) + partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_56) + join (INNER, REPLICATED): + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) + scan catalog_sales local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q79.plan.txt index 7ae16b239b0c..5efde1d3d8c4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q79.plan.txt @@ -2,26 +2,23 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["s_city", "ss_addr_sk", "ss_customer_sk", "ss_ticket_number"]) + partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q80.plan.txt index cbfdaea7dfa3..a4b2da9fdab0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q80.plan.txt @@ -15,12 +15,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -43,12 +41,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_returns + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -71,12 +67,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q81.plan.txt index e4507b49eac7..f07bffd03de9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q81.plan.txt @@ -9,16 +9,14 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state, cr_returning_customer_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -31,24 +29,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state_88) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_88"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state_88) - final aggregation over (ca_state_88, cr_returning_customer_sk_28) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_28"]) - partial aggregation over (ca_state_88, cr_returning_customer_sk_28) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_31"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address + partial aggregation over (ca_state_88) + final aggregation over (ca_state_88, cr_returning_customer_sk_28) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state_88", "cr_returning_customer_sk_28"]) + partial aggregation over (ca_state_88, cr_returning_customer_sk_28) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cr_returning_addr_sk_31"]) + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk_80"]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q82.plan.txt index 4232c710d2d6..532952036fc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q82.plan.txt @@ -3,18 +3,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + partial aggregation over (i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - join (INNER, REPLICATED, can skip output duplicates): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q83.plan.txt index 75ea801d2649..108710add4fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q83.plan.txt @@ -7,34 +7,30 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_6) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_6) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_40) - scan date_dim + join (INNER, REPLICATED): + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_6) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_6"]) + partial aggregation over (d_date_6) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_40) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_40"]) + partial aggregation over (d_week_seq_40) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -42,65 +38,57 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_131) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_131"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_131) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_165) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_165) - scan date_dim + join (INNER, REPLICATED): + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_131) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_131"]) + partial aggregation over (d_date_131) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_165) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_165"]) + partial aggregation over (d_week_seq_165) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item final aggregation over (i_item_id_201) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_201"]) partial aggregation over (i_item_id_201) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_258) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_date_258"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_date_258) - join (INNER, REPLICATED, can skip output duplicates): - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_292) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_week_seq_292) - scan date_dim + join (INNER, REPLICATED): + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_date_258) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_date_258"]) + partial aggregation over (d_date_258) + join (INNER, REPLICATED, can skip output duplicates): + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_292) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_week_seq_292"]) + partial aggregation over (d_week_seq_292) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q85.plan.txt index f5f3ea2e69f3..23eed8958c96 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q85.plan.txt @@ -3,39 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (r_reason_desc) + partial aggregation over (r_reason_desc) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) - join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan reason + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_refunded_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) + join (INNER, REPLICATED): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q86.plan.txt index a3b94e200aa6..cca927926827 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q86.plan.txt @@ -5,14 +5,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (groupid, i_category$gid, i_class$gid) local exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) remote exchange (REPARTITION, HASH, ["groupid", "i_category$gid", "i_class$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (groupid, i_category$gid, i_class$gid) + partial aggregation over (groupid, i_category$gid, i_class$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q87.plan.txt index 8d422b83cadf..9f02b3c87024 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q87.plan.txt @@ -11,16 +11,14 @@ final aggregation over () partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) @@ -28,16 +26,14 @@ final aggregation over () partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) @@ -45,13 +41,11 @@ final aggregation over () partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q89.plan.txt index b5495ae3801b..396d705bbc38 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q89.plan.txt @@ -2,22 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_brand", "i_category", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["d_moy", "i_brand", "i_category", "i_class", "s_company_name", "s_store_name"]) + partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q91.plan.txt index e22013240d22..2d679f5abc8e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q91.plan.txt @@ -4,33 +4,30 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_call_center_id", "cc_manager", "cc_name", "cd_education_status", "cd_marital_status"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) + partial aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q92.plan.txt index 03866eaa7ffd..55eb5a4a778f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q92.plan.txt @@ -7,25 +7,23 @@ final aggregation over () final aggregation over (ws_item_sk_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ws_item_sk_7) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk_7) join (INNER, REPLICATED): scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q93.plan.txt index a97d8ef18468..ca9dc7543db7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q93.plan.txt @@ -7,12 +7,10 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan store_returns + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q94.plan.txt index 7c882fee5529..715050fb629b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q94.plan.txt @@ -7,31 +7,28 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (wr_order_number) - scan web_returns + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_23", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_state", "d_date", "unique", "web_company_name", "ws_ext_ship_cost", "ws_net_profit", "ws_order_number_23", "ws_ship_addr_sk", "ws_ship_date_sk", "ws_warehouse_sk", "ws_web_site_sk"]) + partial aggregation over (ca_state, d_date, unique, web_company_name, ws_ext_ship_cost, ws_net_profit, ws_order_number_23, ws_ship_addr_sk, ws_ship_date_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q95.plan.txt index 21e3f3d023f2..aaf3edd0ed42 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q95.plan.txt @@ -9,41 +9,35 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_101"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_returns + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_137"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales join (INNER, PARTITIONED): final aggregation over (ws_order_number_23) local exchange (GATHER, SINGLE, []) partial aggregation over (ws_order_number_23) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, ["ws_order_number_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_59"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan web_sales + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q97.plan.txt index 24bc0bdee176..7b9c83293534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q97.plan.txt @@ -6,20 +6,18 @@ final aggregation over () final aggregation over (ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (ss_customer_sk, ss_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_customer_sk, ss_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim final aggregation over (cs_bill_customer_sk, cs_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk", "cs_item_sk"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cs_bill_customer_sk, cs_item_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk, cs_item_sk) + join (INNER, REPLICATED, can skip output duplicates): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q98.plan.txt index e5840c437809..9775b6c511f5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q98.plan.txt @@ -3,18 +3,16 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_category", "i_class", "i_current_price", "i_item_desc", "i_item_id"]) + partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + join (INNER, REPLICATED): + join (INNER, REPLICATED): + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q99.plan.txt index ba34f85785e9..e7f593fe04e2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/parquet/unpartitioned/q99.plan.txt @@ -3,22 +3,21 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, sm_type, substr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "sm_type", "substr$gid"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (cc_name, sm_type, substr$gid) + partial aggregation over (cc_name, sm_type, substr$gid) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q02.plan.txt index 73ba5b2465e3..b593422eaaf0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q02.plan.txt @@ -6,43 +6,40 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_23) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_23) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_23) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey"]) join (INNER, REPLICATED): - scan partsupp + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q03.plan.txt index 1c9dd5870ba2..4a9e1323a310 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_5, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q04.plan.txt index ee8edb198516..5d0194bd4f62 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q04.plan.txt @@ -4,15 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, REPLICATED): - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, REPLICATED): + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (orderkey_0) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q05.plan.txt index de2c161dfdfd..cb04cfef04bd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q05.plan.txt @@ -7,27 +7,24 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (name_22) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["nationkey_14", "orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q07.plan.txt index 27c32914ba01..e334b46ac4d6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q07.plan.txt @@ -9,25 +9,22 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan customer + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q08.plan.txt index 4241a644111f..34062f9cadd6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q08.plan.txt @@ -9,35 +9,31 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey"]) join (INNER, REPLICATED): - scan lineitem + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_11"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q09.plan.txt index b05841b29304..8b8f3844d639 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q10.plan.txt index f27e3195bc8a..4b45077ce17c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q10.plan.txt @@ -6,19 +6,16 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (acctbal, address, comment_7, custkey_6, name, name_12, phone) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q15.plan.txt index 99d24e6b9a47..c95f702a6873 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q15.plan.txt @@ -9,9 +9,8 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +20,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_7) - scan lineitem + partial aggregation over (suppkey_7) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q16.plan.txt index 9c0d1a7a90d3..97eea04d0aa1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_5"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q17.plan.txt index 3941a8a3ddc9..0d192b20aa3d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q17.plan.txt @@ -14,9 +14,8 @@ final aggregation over () final aggregation over (partkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_6) - scan lineitem + partial aggregation over (partkey_6) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q18.plan.txt index bb9785189f52..463aec3e5325 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q18.plan.txt @@ -5,25 +5,21 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_5, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over (orderkey_11) - final aggregation over (orderkey_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_11) - scan lineitem + join (INNER, REPLICATED): + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over (orderkey_11) + final aggregation over (orderkey_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_11"]) + partial aggregation over (orderkey_11) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q19.plan.txt index 4ba17d407e65..fa00639798fe 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q20.plan.txt index ada99e1e7958..89b191d4eafd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_21, suppkey_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_21, suppkey_22) - scan lineitem + partial aggregation over (suppkey_6) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_21, suppkey_22) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_6"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_13) - scan part + remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) + partial aggregation over (partkey_21, suppkey_22) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_6"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_13"]) + partial aggregation over (partkey_13) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q21.plan.txt index 8dff250145df..1460eaa40b15 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q21.plan.txt @@ -3,35 +3,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - single aggregation over (commitdate, exists, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique) - join (LEFT, PARTITIONED): - final aggregation over (commitdate, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique_62) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique_62) + partial aggregation over (name) + single aggregation over (commitdate, exists, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique) + join (LEFT, PARTITIONED): + final aggregation over (commitdate, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique_62) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, name, name_12, nationkey, orderkey, orderstatus, receiptdate, suppkey_0, unique_62) + join (INNER, PARTITIONED): join (INNER, PARTITIONED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_38"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + remote exchange (REPARTITION, HASH, ["orderkey_5"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_17"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_38"]) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q22.plan.txt index dd43ccd8d138..0cc72848180d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/partitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_11) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_11) - scan orders + partial aggregation over (custkey_11) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q02.plan.txt index 73ba5b2465e3..b593422eaaf0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q02.plan.txt @@ -6,43 +6,40 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_23) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_23"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_23) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_23) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey"]) join (INNER, REPLICATED): - scan partsupp + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q03.plan.txt index 1c9dd5870ba2..4a9e1323a310 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_5, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q04.plan.txt index 9964bd40d4f8..91dcfd113534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q04.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, PARTITIONED): + final aggregation over (orderkey_0) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q05.plan.txt index 784940efa7ec..11e54014a4ca 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q05.plan.txt @@ -7,27 +7,24 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (name_21) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["nationkey_13", "orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q07.plan.txt index 19e22ce67b58..4a90291c58c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q07.plan.txt @@ -7,27 +7,24 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (expr$gid, name_21, name_27) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q08.plan.txt index 01b05dbfabd0..c196a44d68e0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q08.plan.txt @@ -11,33 +11,29 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey_16"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan lineitem + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q09.plan.txt index 54087e8f540c..24be20eb1056 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_12"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_17"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q10.plan.txt index 2b2a513ac7ca..60df9b930b5b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q10.plan.txt @@ -6,18 +6,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q15.plan.txt index da1195652f8d..02ee343d1df5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q15.plan.txt @@ -4,14 +4,12 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +19,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_7"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_7) - scan lineitem + partial aggregation over (suppkey_7) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q16.plan.txt index 9c0d1a7a90d3..97eea04d0aa1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_5"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q17.plan.txt index 69b972107f29..72a457f937bf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q17.plan.txt @@ -7,17 +7,15 @@ final aggregation over () final aggregation over (partkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_6) - scan lineitem + partial aggregation over (partkey_6) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q18.plan.txt index 2124161181e2..b9b41cc41b90 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q18.plan.txt @@ -5,24 +5,20 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_5, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders single aggregation over (orderkey_10) final aggregation over (orderkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_10) - scan lineitem + partial aggregation over (orderkey_10) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q19.plan.txt index 4ba17d407e65..fa00639798fe 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q20.plan.txt index ada99e1e7958..89b191d4eafd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_21, suppkey_22) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_21, suppkey_22) - scan lineitem + partial aggregation over (suppkey_6) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_21, suppkey_22) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_6"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_13) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_13) - scan part + remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) + partial aggregation over (partkey_21, suppkey_22) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_6"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_13) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_13"]) + partial aggregation over (partkey_13) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q21.plan.txt index e63e68a5a27f..d62c188bfd61 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q21.plan.txt @@ -3,36 +3,31 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - final aggregation over (commitdate, exists, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, exists, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_36"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - final aggregation over (commitdate, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique_59) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique_59) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_16"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + partial aggregation over (name) + final aggregation over (commitdate, exists, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, exists, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_36"]) + scan lineitem + final aggregation over (commitdate, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique_59) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, name, name_11, nationkey, orderkey_16, orderstatus, receiptdate, suppkey_0, unique_59) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_16"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_5"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q22.plan.txt index dd43ccd8d138..0cc72848180d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/hive/unpartitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_11) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_11) - scan orders + partial aggregation over (custkey_11) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q02.plan.txt index dc93bb7bc1fd..358da707a03d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q02.plan.txt @@ -6,45 +6,41 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_19) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_19) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_4"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q03.plan.txt index 4e117498e676..f5cf1c82bd37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q04.plan.txt index 9964bd40d4f8..91dcfd113534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q04.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, PARTITIONED): + final aggregation over (orderkey_0) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q05.plan.txt index 5ad7aaebee00..8d476b717a72 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q05.plan.txt @@ -9,27 +9,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q07.plan.txt index 727ba23e2f4b..64d5f1fecd08 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q07.plan.txt @@ -9,25 +9,22 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q08.plan.txt index 724fc473a19a..88f8a11f4445 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q08.plan.txt @@ -12,32 +12,28 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey_13"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan lineitem + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q09.plan.txt index 935c5e847127..a55a51f232bd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q10.plan.txt index c084f568796c..7ab0ad56a3f1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q10.plan.txt @@ -6,18 +6,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q15.plan.txt index 8d72d43209bf..b9a65952a467 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q15.plan.txt @@ -4,14 +4,12 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +19,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - scan lineitem + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q16.plan.txt index c94f3d6e659d..bd14aff914fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_4"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q17.plan.txt index 1c0100d18830..30539653b56b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q17.plan.txt @@ -7,17 +7,15 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_5) - scan lineitem + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q18.plan.txt index 741668695fd3..296a79e4c9cf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q18.plan.txt @@ -9,20 +9,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_8) - scan lineitem + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q19.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q20.plan.txt index 78592280adc3..cb79004dac3a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_5) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_18, suppkey_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + partial aggregation over (suppkey_5) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_18, suppkey_19) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_11) - scan part + remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) + partial aggregation over (partkey_18, suppkey_19) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_11"]) + partial aggregation over (partkey_11) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q21.plan.txt index fdc46b4f0e52..313eb576ce37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q21.plan.txt @@ -3,35 +3,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - single aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - join (LEFT, PARTITIONED): - final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (name) + single aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + join (LEFT, PARTITIONED): + final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_32"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_13"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_4"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q22.plan.txt index 12c1be1b0be3..eba30d3c79d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/partitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_10) - scan orders + partial aggregation over (custkey_10) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q02.plan.txt index dc93bb7bc1fd..358da707a03d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q02.plan.txt @@ -6,45 +6,41 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_19) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_19) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_4"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q03.plan.txt index 4e117498e676..f5cf1c82bd37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q04.plan.txt index 9964bd40d4f8..91dcfd113534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q04.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, PARTITIONED): + final aggregation over (orderkey_0) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q05.plan.txt index 5ad7aaebee00..8d476b717a72 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q05.plan.txt @@ -9,27 +9,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q07.plan.txt index 727ba23e2f4b..64d5f1fecd08 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q07.plan.txt @@ -9,25 +9,22 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q08.plan.txt index 724fc473a19a..88f8a11f4445 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q08.plan.txt @@ -12,32 +12,28 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey_13"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan lineitem + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q09.plan.txt index 935c5e847127..a55a51f232bd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q10.plan.txt index c084f568796c..7ab0ad56a3f1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q10.plan.txt @@ -6,18 +6,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q15.plan.txt index 8d72d43209bf..b9a65952a467 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q15.plan.txt @@ -4,14 +4,12 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +19,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - scan lineitem + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q16.plan.txt index c94f3d6e659d..bd14aff914fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_4"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q17.plan.txt index 1c0100d18830..30539653b56b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q17.plan.txt @@ -7,17 +7,15 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_5) - scan lineitem + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q18.plan.txt index 741668695fd3..296a79e4c9cf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q18.plan.txt @@ -9,20 +9,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_8) - scan lineitem + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q19.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q20.plan.txt index 78592280adc3..cb79004dac3a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_5) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_18, suppkey_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + partial aggregation over (suppkey_5) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_18, suppkey_19) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_11) - scan part + remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) + partial aggregation over (partkey_18, suppkey_19) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_11"]) + partial aggregation over (partkey_11) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q21.plan.txt index fdc46b4f0e52..313eb576ce37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q21.plan.txt @@ -3,35 +3,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - single aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - join (LEFT, PARTITIONED): - final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (name) + single aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + join (LEFT, PARTITIONED): + final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_32"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_13"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_4"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q22.plan.txt index 12c1be1b0be3..eba30d3c79d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/orc/unpartitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_10) - scan orders + partial aggregation over (custkey_10) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q02.plan.txt index 5b26d6ebbd8a..075fb157c432 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q02.plan.txt @@ -6,43 +6,40 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_19) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_19) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_4"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey"]) join (INNER, REPLICATED): - scan partsupp + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q03.plan.txt index 4e117498e676..f5cf1c82bd37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q04.plan.txt index 9964bd40d4f8..91dcfd113534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q04.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, PARTITIONED): + final aggregation over (orderkey_0) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q05.plan.txt index 5ad7aaebee00..8d476b717a72 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q05.plan.txt @@ -9,27 +9,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q07.plan.txt index 727ba23e2f4b..64d5f1fecd08 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q07.plan.txt @@ -9,25 +9,22 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q08.plan.txt index 724fc473a19a..88f8a11f4445 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q08.plan.txt @@ -12,32 +12,28 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey_13"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan lineitem + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q09.plan.txt index 935c5e847127..a55a51f232bd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q10.plan.txt index c084f568796c..7ab0ad56a3f1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q10.plan.txt @@ -6,18 +6,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q15.plan.txt index 8d72d43209bf..b9a65952a467 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q15.plan.txt @@ -4,14 +4,12 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +19,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - scan lineitem + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q16.plan.txt index c94f3d6e659d..bd14aff914fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_4"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q17.plan.txt index 1c0100d18830..30539653b56b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q17.plan.txt @@ -7,17 +7,15 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_5) - scan lineitem + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q18.plan.txt index 1fef6bf01674..65c3835cf86a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q18.plan.txt @@ -5,24 +5,20 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_4, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_8) - scan lineitem + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q19.plan.txt index 4ba17d407e65..fa00639798fe 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q20.plan.txt index 78592280adc3..cb79004dac3a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_5) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_18, suppkey_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + partial aggregation over (suppkey_5) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_18, suppkey_19) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_11) - scan part + remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) + partial aggregation over (partkey_18, suppkey_19) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_11"]) + partial aggregation over (partkey_11) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q21.plan.txt index d6d9cc304775..7d0cc36c8856 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q21.plan.txt @@ -3,36 +3,31 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - final aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_32"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + partial aggregation over (name) + final aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + scan lineitem + final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_13"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_4"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q22.plan.txt index 12c1be1b0be3..eba30d3c79d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/partitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_10) - scan orders + partial aggregation over (custkey_10) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q01.plan.txt index 72eae868fa59..d632b41b395a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q01.plan.txt @@ -4,6 +4,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (linestatus, returnflag) - scan lineitem + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q02.plan.txt index 5b26d6ebbd8a..075fb157c432 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q02.plan.txt @@ -6,43 +6,40 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_19) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + partial aggregation over (partkey_19) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_4"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey"]) join (INNER, REPLICATED): - scan partsupp + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q03.plan.txt index 4e117498e676..f5cf1c82bd37 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q03.plan.txt @@ -5,15 +5,12 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q04.plan.txt index 9964bd40d4f8..91dcfd113534 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q04.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_0) - scan lineitem + partial aggregation over (orderpriority) + join (INNER, PARTITIONED): + final aggregation over (orderkey_0) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey_0"]) + partial aggregation over (orderkey_0) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q05.plan.txt index 5ad7aaebee00..8d476b717a72 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q05.plan.txt @@ -9,27 +9,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q07.plan.txt index 727ba23e2f4b..64d5f1fecd08 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q07.plan.txt @@ -9,25 +9,22 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q08.plan.txt index 724fc473a19a..88f8a11f4445 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q08.plan.txt @@ -12,32 +12,28 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey_13"]) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - scan lineitem + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + scan region local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q09.plan.txt index 935c5e847127..a55a51f232bd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q09.plan.txt @@ -13,24 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q10.plan.txt index c084f568796c..7ab0ad56a3f1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q10.plan.txt @@ -6,18 +6,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q11.plan.txt index 70b8b0f9b94c..bfc5af4b5bb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q11.plan.txt @@ -5,17 +5,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey) - join (INNER, REPLICATED): - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + partial aggregation over (partkey) + join (INNER, REPLICATED): + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q12.plan.txt index 36ac0385529a..7fb5b836dfa9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q12.plan.txt @@ -7,9 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q13.plan.txt index c277ff8900bc..e30665fcb33b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q13.plan.txt @@ -4,16 +4,13 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (count) - final aggregation over (custkey) - local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["custkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + partial aggregation over (count) + final aggregation over (custkey) + local exchange (GATHER, SINGLE, []) + partial aggregation over (custkey) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["custkey_0"]) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["custkey"]) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q14.plan.txt index d2330a52a902..e46c0418a759 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q14.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q15.plan.txt index 8d72d43209bf..b9a65952a467 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q15.plan.txt @@ -4,14 +4,12 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_0) - scan lineitem + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -21,6 +19,5 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_6) - scan lineitem + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q16.plan.txt index c94f3d6e659d..bd14aff914fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q16.plan.txt @@ -4,22 +4,18 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (brand, size, type) - final aggregation over (brand, size, suppkey, type) - local exchange (GATHER, SINGLE, []) - partial aggregation over (brand, size, suppkey, type) - semijoin (PARTITIONED): - remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["suppkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan supplier + partial aggregation over (brand, size, type) + final aggregation over (brand, size, suppkey, type) + local exchange (GATHER, SINGLE, []) + partial aggregation over (brand, size, suppkey, type) + semijoin (PARTITIONED): + remote exchange (REPARTITION, HASH, ["suppkey"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_0"]) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["suppkey_4"]) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q17.plan.txt index 1c0100d18830..30539653b56b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q17.plan.txt @@ -7,17 +7,15 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_5) - scan lineitem + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q18.plan.txt index 1fef6bf01674..65c3835cf86a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q18.plan.txt @@ -5,24 +5,20 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_4, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (orderkey_8) - scan lineitem + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan customer + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q19.plan.txt index 4ba17d407e65..fa00639798fe 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q19.plan.txt @@ -4,9 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan part + scan part diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q20.plan.txt index 78592280adc3..cb79004dac3a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q20.plan.txt @@ -5,37 +5,32 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (suppkey_5) - cross join (can skip output duplicates): - join (RIGHT, PARTITIONED, can skip output duplicates): - final aggregation over (partkey_18, suppkey_19) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + partial aggregation over (suppkey_5) + cross join (can skip output duplicates): + join (RIGHT, PARTITIONED, can skip output duplicates): + final aggregation over (partkey_18, suppkey_19) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) - join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["partkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan partsupp - final aggregation over (partkey_11) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_11"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (partkey_11) - scan part + remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) + partial aggregation over (partkey_18, suppkey_19) + scan lineitem local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - single aggregation over () - values (1 rows) + remote exchange (REPARTITION, HASH, ["partkey", "suppkey_5"]) + join (INNER, PARTITIONED, can skip output duplicates): + remote exchange (REPARTITION, HASH, ["partkey"]) + scan partsupp + final aggregation over (partkey_11) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["partkey_11"]) + partial aggregation over (partkey_11) + scan part + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q21.plan.txt index d6d9cc304775..7d0cc36c8856 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q21.plan.txt @@ -3,36 +3,31 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (name) - final aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) - join (RIGHT, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_32"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey_13"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan lineitem - local exchange (GATHER, SINGLE, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["orderkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_4"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - scan orders + partial aggregation over (name) + final aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, exists, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique) + join (RIGHT, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + scan lineitem + final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + local exchange (GATHER, SINGLE, []) + partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey_13"]) + scan lineitem + local exchange (GATHER, SINGLE, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["orderkey"]) + join (INNER, REPLICATED): + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + scan supplier + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan nation + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["orderkey_4"]) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q22.plan.txt index 12c1be1b0be3..eba30d3c79d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/parquet/unpartitioned/q22.plan.txt @@ -7,19 +7,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - cross join: - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over () - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - partial aggregation over () - scan customer + cross join: + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over () + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + partial aggregation over () + scan customer final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - local exchange (REPARTITION, ROUND_ROBIN, []) - partial aggregation over (custkey_10) - scan orders + partial aggregation over (custkey_10) + scan orders