From 9284477fdaa0a6c3655248021d7dba222dcb2092 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Wed, 24 Aug 2022 14:09:25 -0400 Subject: [PATCH 1/5] Ensure PartitionedOutputOperator is run with fixed local distribution PartitionedOutputOperator maintains buffers for each output partition. When the operator is run in the same pipeline as the TableScanOperator the buffers are flushed after each split resulting in small pages being created. --- .../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 +- .../tpcds/iceberg/partitioned/q01.plan.txt | 51 +-- .../tpcds/iceberg/partitioned/q02.plan.txt | 67 ++-- .../tpcds/iceberg/partitioned/q03.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q04.plan.txt | 180 +++++----- .../tpcds/iceberg/partitioned/q05.plan.txt | 6 +- .../tpcds/iceberg/partitioned/q06.plan.txt | 43 ++- .../tpcds/iceberg/partitioned/q07.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q08.plan.txt | 43 ++- .../tpcds/iceberg/partitioned/q10.plan.txt | 84 ++--- .../tpcds/iceberg/partitioned/q11.plan.txt | 120 ++++--- .../tpcds/iceberg/partitioned/q12.plan.txt | 31 +- .../tpcds/iceberg/partitioned/q13.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q14.plan.txt | 324 +++++++++--------- .../tpcds/iceberg/partitioned/q15.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q16.plan.txt | 43 +-- .../tpcds/iceberg/partitioned/q17.plan.txt | 46 +-- .../tpcds/iceberg/partitioned/q18.plan.txt | 52 +-- .../tpcds/iceberg/partitioned/q19.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q20.plan.txt | 28 +- .../tpcds/iceberg/partitioned/q21.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q23.plan.txt | 140 ++++---- .../tpcds/iceberg/partitioned/q24.plan.txt | 75 ++-- .../tpcds/iceberg/partitioned/q25.plan.txt | 44 +-- .../tpcds/iceberg/partitioned/q26.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q27.plan.txt | 28 +- .../tpcds/iceberg/partitioned/q28.plan.txt | 18 +- .../tpcds/iceberg/partitioned/q29.plan.txt | 44 +-- .../tpcds/iceberg/partitioned/q30.plan.txt | 75 ++-- .../tpcds/iceberg/partitioned/q31.plan.txt | 151 ++++---- .../tpcds/iceberg/partitioned/q32.plan.txt | 24 +- .../tpcds/iceberg/partitioned/q33.plan.txt | 102 +++--- .../tpcds/iceberg/partitioned/q34.plan.txt | 33 +- .../tpcds/iceberg/partitioned/q35.plan.txt | 46 +-- .../tpcds/iceberg/partitioned/q36.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q37.plan.txt | 23 +- .../tpcds/iceberg/partitioned/q38.plan.txt | 42 ++- .../tpcds/iceberg/partitioned/q39.plan.txt | 60 ++-- .../tpcds/iceberg/partitioned/q40.plan.txt | 6 +- .../tpcds/iceberg/partitioned/q41.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q42.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q43.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q44.plan.txt | 60 ++-- .../tpcds/iceberg/partitioned/q45.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q46.plan.txt | 45 +-- .../tpcds/iceberg/partitioned/q47.plan.txt | 90 ++--- .../tpcds/iceberg/partitioned/q48.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q49.plan.txt | 111 +++--- .../tpcds/iceberg/partitioned/q50.plan.txt | 14 +- .../tpcds/iceberg/partitioned/q51.plan.txt | 51 +-- .../tpcds/iceberg/partitioned/q52.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q53.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q54.plan.txt | 117 ++++--- .../tpcds/iceberg/partitioned/q55.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q56.plan.txt | 102 +++--- .../tpcds/iceberg/partitioned/q57.plan.txt | 90 ++--- .../tpcds/iceberg/partitioned/q58.plan.txt | 123 ++++--- .../tpcds/iceberg/partitioned/q59.plan.txt | 63 ++-- .../tpcds/iceberg/partitioned/q60.plan.txt | 99 +++--- .../tpcds/iceberg/partitioned/q61.plan.txt | 52 +-- .../tpcds/iceberg/partitioned/q62.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q63.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q64.plan.txt | 318 +++++++++-------- .../tpcds/iceberg/partitioned/q65.plan.txt | 38 +- .../tpcds/iceberg/partitioned/q66.plan.txt | 68 ++-- .../tpcds/iceberg/partitioned/q68.plan.txt | 45 +-- .../tpcds/iceberg/partitioned/q69.plan.txt | 63 ++-- .../tpcds/iceberg/partitioned/q70.plan.txt | 28 +- .../tpcds/iceberg/partitioned/q72.plan.txt | 32 +- .../tpcds/iceberg/partitioned/q73.plan.txt | 33 +- .../tpcds/iceberg/partitioned/q74.plan.txt | 120 ++++--- .../tpcds/iceberg/partitioned/q75.plan.txt | 198 ++++++----- .../tpcds/iceberg/partitioned/q76.plan.txt | 38 +- .../tpcds/iceberg/partitioned/q77.plan.txt | 86 ++--- .../tpcds/iceberg/partitioned/q78.plan.txt | 88 ++--- .../tpcds/iceberg/partitioned/q79.plan.txt | 33 +- .../tpcds/iceberg/partitioned/q80.plan.txt | 18 +- .../tpcds/iceberg/partitioned/q81.plan.txt | 75 ++-- .../tpcds/iceberg/partitioned/q82.plan.txt | 23 +- .../tpcds/iceberg/partitioned/q83.plan.txt | 144 ++++---- .../tpcds/iceberg/partitioned/q85.plan.txt | 65 ++-- .../tpcds/iceberg/partitioned/q86.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q87.plan.txt | 42 ++- .../tpcds/iceberg/partitioned/q89.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q91.plan.txt | 28 +- .../tpcds/iceberg/partitioned/q92.plan.txt | 24 +- .../tpcds/iceberg/partitioned/q93.plan.txt | 6 +- .../tpcds/iceberg/partitioned/q94.plan.txt | 43 +-- .../tpcds/iceberg/partitioned/q95.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q97.plan.txt | 26 +- .../tpcds/iceberg/partitioned/q98.plan.txt | 28 +- .../tpcds/iceberg/partitioned/q99.plan.txt | 19 +- .../tpcds/iceberg/unpartitioned/q01.plan.txt | 51 +-- .../tpcds/iceberg/unpartitioned/q02.plan.txt | 67 ++-- .../tpcds/iceberg/unpartitioned/q03.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q04.plan.txt | 180 +++++----- .../tpcds/iceberg/unpartitioned/q05.plan.txt | 6 +- .../tpcds/iceberg/unpartitioned/q06.plan.txt | 43 ++- .../tpcds/iceberg/unpartitioned/q07.plan.txt | 19 +- .../tpcds/iceberg/unpartitioned/q08.plan.txt | 43 ++- .../tpcds/iceberg/unpartitioned/q10.plan.txt | 84 ++--- .../tpcds/iceberg/unpartitioned/q11.plan.txt | 120 ++++--- .../tpcds/iceberg/unpartitioned/q12.plan.txt | 31 +- .../tpcds/iceberg/unpartitioned/q13.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q14.plan.txt | 324 +++++++++--------- .../tpcds/iceberg/unpartitioned/q15.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q16.plan.txt | 43 +-- .../tpcds/iceberg/unpartitioned/q17.plan.txt | 46 +-- .../tpcds/iceberg/unpartitioned/q18.plan.txt | 52 +-- .../tpcds/iceberg/unpartitioned/q19.plan.txt | 19 +- .../tpcds/iceberg/unpartitioned/q20.plan.txt | 28 +- .../tpcds/iceberg/unpartitioned/q21.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q23.plan.txt | 140 ++++---- .../tpcds/iceberg/unpartitioned/q24.plan.txt | 75 ++-- .../tpcds/iceberg/unpartitioned/q25.plan.txt | 44 +-- .../tpcds/iceberg/unpartitioned/q26.plan.txt | 19 +- .../tpcds/iceberg/unpartitioned/q27.plan.txt | 28 +- .../tpcds/iceberg/unpartitioned/q28.plan.txt | 18 +- .../tpcds/iceberg/unpartitioned/q29.plan.txt | 44 +-- .../tpcds/iceberg/unpartitioned/q30.plan.txt | 75 ++-- .../tpcds/iceberg/unpartitioned/q31.plan.txt | 151 ++++---- .../tpcds/iceberg/unpartitioned/q32.plan.txt | 24 +- .../tpcds/iceberg/unpartitioned/q33.plan.txt | 102 +++--- .../tpcds/iceberg/unpartitioned/q34.plan.txt | 33 +- .../tpcds/iceberg/unpartitioned/q35.plan.txt | 46 +-- .../tpcds/iceberg/unpartitioned/q36.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q37.plan.txt | 23 +- .../tpcds/iceberg/unpartitioned/q38.plan.txt | 42 ++- .../tpcds/iceberg/unpartitioned/q39.plan.txt | 60 ++-- .../tpcds/iceberg/unpartitioned/q40.plan.txt | 6 +- .../tpcds/iceberg/unpartitioned/q41.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q42.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q43.plan.txt | 30 +- .../tpcds/iceberg/unpartitioned/q44.plan.txt | 60 ++-- .../tpcds/iceberg/unpartitioned/q45.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q46.plan.txt | 45 +-- .../tpcds/iceberg/unpartitioned/q47.plan.txt | 90 ++--- .../tpcds/iceberg/unpartitioned/q48.plan.txt | 17 +- .../tpcds/iceberg/unpartitioned/q49.plan.txt | 111 +++--- .../tpcds/iceberg/unpartitioned/q50.plan.txt | 14 +- .../tpcds/iceberg/unpartitioned/q51.plan.txt | 51 +-- .../tpcds/iceberg/unpartitioned/q52.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q53.plan.txt | 30 +- .../tpcds/iceberg/unpartitioned/q54.plan.txt | 117 ++++--- .../tpcds/iceberg/unpartitioned/q55.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q56.plan.txt | 102 +++--- .../tpcds/iceberg/unpartitioned/q57.plan.txt | 90 ++--- .../tpcds/iceberg/unpartitioned/q58.plan.txt | 123 ++++--- .../tpcds/iceberg/unpartitioned/q59.plan.txt | 63 ++-- .../tpcds/iceberg/unpartitioned/q60.plan.txt | 99 +++--- .../tpcds/iceberg/unpartitioned/q61.plan.txt | 52 +-- .../tpcds/iceberg/unpartitioned/q62.plan.txt | 19 +- .../tpcds/iceberg/unpartitioned/q63.plan.txt | 30 +- .../tpcds/iceberg/unpartitioned/q64.plan.txt | 318 +++++++++-------- .../tpcds/iceberg/unpartitioned/q65.plan.txt | 38 +- .../tpcds/iceberg/unpartitioned/q66.plan.txt | 68 ++-- .../tpcds/iceberg/unpartitioned/q68.plan.txt | 45 +-- .../tpcds/iceberg/unpartitioned/q69.plan.txt | 63 ++-- .../tpcds/iceberg/unpartitioned/q70.plan.txt | 28 +- .../tpcds/iceberg/unpartitioned/q72.plan.txt | 32 +- .../tpcds/iceberg/unpartitioned/q73.plan.txt | 33 +- .../tpcds/iceberg/unpartitioned/q74.plan.txt | 120 ++++--- .../tpcds/iceberg/unpartitioned/q75.plan.txt | 198 ++++++----- .../tpcds/iceberg/unpartitioned/q76.plan.txt | 38 +- .../tpcds/iceberg/unpartitioned/q77.plan.txt | 86 ++--- .../tpcds/iceberg/unpartitioned/q78.plan.txt | 88 ++--- .../tpcds/iceberg/unpartitioned/q79.plan.txt | 33 +- .../tpcds/iceberg/unpartitioned/q80.plan.txt | 18 +- .../tpcds/iceberg/unpartitioned/q81.plan.txt | 75 ++-- .../tpcds/iceberg/unpartitioned/q82.plan.txt | 23 +- .../tpcds/iceberg/unpartitioned/q83.plan.txt | 144 ++++---- .../tpcds/iceberg/unpartitioned/q85.plan.txt | 65 ++-- .../tpcds/iceberg/unpartitioned/q86.plan.txt | 15 +- .../tpcds/iceberg/unpartitioned/q87.plan.txt | 42 ++- .../tpcds/iceberg/unpartitioned/q89.plan.txt | 30 +- .../tpcds/iceberg/unpartitioned/q91.plan.txt | 28 +- .../tpcds/iceberg/unpartitioned/q92.plan.txt | 24 +- .../tpcds/iceberg/unpartitioned/q93.plan.txt | 6 +- .../tpcds/iceberg/unpartitioned/q94.plan.txt | 43 +-- .../tpcds/iceberg/unpartitioned/q95.plan.txt | 30 +- .../tpcds/iceberg/unpartitioned/q97.plan.txt | 26 +- .../tpcds/iceberg/unpartitioned/q98.plan.txt | 28 +- .../tpcds/iceberg/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/partitioned/q01.plan.txt | 5 +- .../tpch/iceberg/partitioned/q02.plan.txt | 54 +-- .../tpch/iceberg/partitioned/q03.plan.txt | 9 +- .../tpch/iceberg/partitioned/q04.plan.txt | 21 +- .../tpch/iceberg/partitioned/q05.plan.txt | 28 +- .../tpch/iceberg/partitioned/q07.plan.txt | 33 +- .../tpch/iceberg/partitioned/q08.plan.txt | 34 +- .../tpch/iceberg/partitioned/q09.plan.txt | 15 +- .../tpch/iceberg/partitioned/q10.plan.txt | 9 +- .../tpch/iceberg/partitioned/q11.plan.txt | 21 +- .../tpch/iceberg/partitioned/q12.plan.txt | 6 +- .../tpch/iceberg/partitioned/q13.plan.txt | 23 +- .../tpch/iceberg/partitioned/q14.plan.txt | 6 +- .../tpch/iceberg/partitioned/q15.plan.txt | 13 +- .../tpch/iceberg/partitioned/q16.plan.txt | 34 +- .../tpch/iceberg/partitioned/q17.plan.txt | 16 +- .../tpch/iceberg/partitioned/q18.plan.txt | 14 +- .../tpch/iceberg/partitioned/q19.plan.txt | 6 +- .../tpch/iceberg/partitioned/q20.plan.txt | 55 +-- .../tpch/iceberg/partitioned/q21.plan.txt | 57 +-- .../tpch/iceberg/partitioned/q22.plan.txt | 24 +- .../tpch/iceberg/unpartitioned/q01.plan.txt | 5 +- .../tpch/iceberg/unpartitioned/q02.plan.txt | 54 +-- .../tpch/iceberg/unpartitioned/q03.plan.txt | 9 +- .../tpch/iceberg/unpartitioned/q04.plan.txt | 21 +- .../tpch/iceberg/unpartitioned/q05.plan.txt | 28 +- .../tpch/iceberg/unpartitioned/q07.plan.txt | 33 +- .../tpch/iceberg/unpartitioned/q08.plan.txt | 34 +- .../tpch/iceberg/unpartitioned/q09.plan.txt | 15 +- .../tpch/iceberg/unpartitioned/q10.plan.txt | 9 +- .../tpch/iceberg/unpartitioned/q11.plan.txt | 21 +- .../tpch/iceberg/unpartitioned/q12.plan.txt | 6 +- .../tpch/iceberg/unpartitioned/q13.plan.txt | 23 +- .../tpch/iceberg/unpartitioned/q14.plan.txt | 6 +- .../tpch/iceberg/unpartitioned/q15.plan.txt | 13 +- .../tpch/iceberg/unpartitioned/q16.plan.txt | 34 +- .../tpch/iceberg/unpartitioned/q17.plan.txt | 16 +- .../tpch/iceberg/unpartitioned/q18.plan.txt | 14 +- .../tpch/iceberg/unpartitioned/q19.plan.txt | 6 +- .../tpch/iceberg/unpartitioned/q20.plan.txt | 55 +-- .../tpch/iceberg/unpartitioned/q21.plan.txt | 57 +-- .../tpch/iceberg/unpartitioned/q22.plan.txt | 24 +- 467 files changed, 12714 insertions(+), 10515 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 625d7b91880b..149763ea3675 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -175,6 +175,7 @@ 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"; private final List> sessionProperties; @@ -865,6 +866,11 @@ public SystemSessionProperties( FAULT_TOLERANT_EXECUTION_EVENT_DRIVEN_SCHEDULER_ENABLED, "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)); } @@ -1548,4 +1554,9 @@ 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); + } } 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 faf821a80da4..164ee812236f 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,6 +89,8 @@ public class OptimizerConfig private double adaptivePartialAggregationUniqueRowsRatioThreshold = 0.8; private long joinPartitionedBuildMinRowCount = 1_000_000L; + private boolean forceFixedDistributionForPartitionedOutputOperatorEnabled = true; + public enum JoinReorderingStrategy { NONE, @@ -756,4 +758,16 @@ 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 a666f6daeec5..3926d4805561 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,25 +104,33 @@ public class AddExchangesBelowPartialAggregationOverGroupIdRuleSet typeOf(ExchangeNode.class) .with(scope().equalTo(REMOTE)) .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) + typeOf(ProjectNode.class) .with(source().matching( - typeOf(AggregationNode.class).capturedAs(AGGREGATION) - .with(step().equalTo(AggregationNode.Step.PARTIAL)) - .with(nonEmpty(groupingColumns())) + typeOf(ExchangeNode.class) + .with(scope().equalTo(LOCAL)) .with(source().matching( - typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))))); + // 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))))))))))); 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(AggregationNode.class).capturedAs(AGGREGATION) - .with(step().equalTo(AggregationNode.Step.PARTIAL)) - .with(nonEmpty(groupingColumns())) + typeOf(ExchangeNode.class) + .with(scope().equalTo(LOCAL)) .with(source().matching( - typeOf(GroupIdNode.class).capturedAs(GROUP_ID))))); + typeOf(AggregationNode.class).capturedAs(AGGREGATION) + .with(step().equalTo(AggregationNode.Step.PARTIAL)) + .with(nonEmpty(groupingColumns())) + .with(source().matching( + 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 6c3ae07e99c9..e7291d355e46 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,6 +80,7 @@ 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; @@ -743,6 +744,9 @@ 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 f285cf5a408f..1d4bc4718ef0 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,6 +34,7 @@ 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; @@ -248,4 +249,11 @@ 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 12306998472d..6e2b5e791e95 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,7 +89,8 @@ public void testDefaults() .setAdaptivePartialAggregationMinRows(100_000) .setAdaptivePartialAggregationUniqueRowsRatioThreshold(0.8) .setJoinPartitionedBuildMinRowCount(1_000_000) - .setUseExactPartitioning(false)); + .setUseExactPartitioning(false) + .setForceFixedDistributionForPartitionedOutputOperatorEnabled(true)); } @Test @@ -147,6 +148,7 @@ 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() @@ -200,7 +202,8 @@ public void testExplicitPropertyMappings() .setAdaptivePartialAggregationMinRows(1) .setAdaptivePartialAggregationUniqueRowsRatioThreshold(0.99) .setJoinPartitionedBuildMinRowCount(1) - .setUseExactPartitioning(true); + .setUseExactPartitioning(true) + .setForceFixedDistributionForPartitionedOutputOperatorEnabled(false); 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 f87227b16042..40335f3730fd 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,7 +31,6 @@ 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; @@ -73,12 +72,12 @@ public TestAddDynamicFilterSource() JOIN_REORDERING_STRATEGY, JoinReorderingStrategy.NONE.name())); } - @Test(dataProvider = "joinDistributionTypes") - public void testInnerJoin(JoinDistributionType joinDistributionType) + @Test + public void testPartitionedInnerJoin() { assertDistributedPlan( "SELECT l.suppkey FROM lineitem l, supplier s WHERE l.suppkey = s.suppkey", - withJoinDistributionType(joinDistributionType), + withJoinDistributionType(PARTITIONED), anyTree( join(INNER, builder -> builder .equiCriteria("LINEITEM_SK", "SUPPLIER_SK") @@ -94,41 +93,99 @@ public void testInnerJoin(JoinDistributionType joinDistributionType) LOCAL, exchange( REMOTE, - joinDistributionType == PARTITIONED ? REPARTITION : REPLICATE, + 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, node( DynamicFilterSourceNode.class, project( tableScan("supplier", ImmutableMap.of("SUPPLIER_SK", "suppkey")))))))))); } - @Test(dataProvider = "joinDistributionTypes") - public void testSemiJoin(JoinDistributionType joinDistributionType) + @Test + public void testPartitionedSemiJoin() { - 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(joinDistributionType), + noSemiJoinRewrite(PARTITIONED), anyTree( filter("S", project( - semiJoin("X", "Y", "S", Optional.of(semiJoinDistributionType), Optional.of(true), + semiJoin("X", "Y", "S", Optional.of(SemiJoinNode.DistributionType.PARTITIONED), Optional.of(true), anyTree( node( FilterNode.class, tableScan("orders", ImmutableMap.of("X", "orderkey"))) - .with(numberOfDynamicFilters(1))), + .with(numberOfDynamicFilters(1))), + exchange( + LOCAL, exchange( - LOCAL, - exchange( - REMOTE, - joinDistributionType == PARTITIONED ? REPARTITION : REPLICATE, - node( - DynamicFilterSourceNode.class, + REMOTE, + REPARTITION, + node( + DynamicFilterSourceNode.class, + exchange( + LOCAL, + REPARTITION, 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")))))))))))); } @@ -183,8 +240,8 @@ public void testInnerJoinWithUnionAllOnBuild() ImmutableSet.of(), Optional.empty(), ImmutableList.of("SUPPLIER_SK"), - exchange(project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_1", "suppkey")))), - exchange(project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_2", "suppkey"))))))))); + exchange(exchange(LOCAL, project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_1", "suppkey"))))), + exchange(exchange(LOCAL, project(tableScan("supplier", ImmutableMap.of("SUPPLIER_SK_2", "suppkey")))))))))); } @Test @@ -243,8 +300,11 @@ public void testJoinWithPrePartitionedBuild() exchange( REMOTE, REPARTITION, - project( - tableScan("lineitem", ImmutableMap.of("LINEITEM_SK", "suppkey"))))) + exchange( + LOCAL, + REPARTITION, + project( + tableScan("lineitem", ImmutableMap.of("LINEITEM_SK", "suppkey")))))) .right( anyTree( tableScan("supplier", ImmutableMap.of("SUPPLIER_SK", "suppkey"))))))); @@ -269,12 +329,6 @@ 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 75ba685a9c8b..9102f9ada5e1 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,7 +153,8 @@ private void testInsertWithRequiredPartitioning(Session session) node(TableWriterNode.class, exchange(LOCAL, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), - values("column1", "column2")) + exchange(LOCAL, REPARTITION, + values("column1", "column2"))) .with(exchangeWithoutSystemPartitioning())) .with(exchangeWithoutSystemPartitioning())))); } @@ -215,7 +216,8 @@ private void testCreateTableAsSelectWithRequiredPartitioning(Session session) node(TableWriterNode.class, exchange(LOCAL, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("column1"), - values("column1", "column2")) + exchange(LOCAL, REPARTITION, + 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 ef5d17a2ddf9..b5c4671fc165 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,10 +224,11 @@ public void testAggregation() FINAL, exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - aggregation( - ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), - PARTIAL, - anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice"))))))))); + exchange(LOCAL, REPARTITION, + 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", @@ -237,10 +238,11 @@ public void testAggregation() FINAL, exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - aggregation( - ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), - PARTIAL, - anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice"))))))))); + exchange(LOCAL, REPARTITION, + aggregation( + ImmutableMap.of("partial_sum", functionCall("sum", ImmutableList.of("totalprice"))), + PARTIAL, + anyTree(tableScan("orders", ImmutableMap.of("totalprice", "totalprice")))))))))); } @Test @@ -1251,7 +1253,8 @@ public void testUsesDistributedJoinIfNaturallyPartitionedOnProbeSymbols() .right( anyTree( exchange(REMOTE, REPARTITION, - tableScan("region", ImmutableMap.of("RIGHT_REGIONKEY", "regionkey"))))))), + exchange(LOCAL, REPARTITION, + 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( @@ -1749,11 +1752,13 @@ public void testRedundantHashRemovalForUnionAllAndMarkDistinct() node(MarkDistinctNode.class, exchange(LOCAL, REPARTITION, exchange(REMOTE, 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")))), + 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"))))), exchange(REMOTE, REPARTITION, - node(ProjectNode.class, - node(TableScanNode.class)))))))))); + exchange(LOCAL, REPARTITION, + node(ProjectNode.class, + node(TableScanNode.class))))))))))); } @Test @@ -1930,13 +1935,16 @@ public void testGroupingSetsWithDefaultValue() exchange( REMOTE, REPARTITION, - aggregation( - ImmutableMap.of("partial_count", functionCall("count", ImmutableList.of("CONSTANT"))), - PARTIAL, - anyTree( - project( - ImmutableMap.of("CONSTANT", expression("1")), - tableScan("orders")))))))))); + exchange( + LOCAL, + REPARTITION, + 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 7eddc23d01fa..9b2ca0855b95 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,9 +187,10 @@ void assertTableScanPlannedWithoutPartitioning(Session session, String table) aggregation(ImmutableMap.of("COUNT", functionCall("count", ImmutableList.of("COUNT_PART"))), FINAL, exchange(LOCAL, REPARTITION, exchange(REMOTE, REPARTITION, - project( - aggregation(ImmutableMap.of("COUNT_PART", functionCall("count", ImmutableList.of("B"))), PARTIAL, - tableScan(table, ImmutableMap.of("A", "column_a", "B", "column_b"))))))))); + 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")))))))))); 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 1418e9943d28..9b310bdc6d76 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,13 +167,15 @@ public void testRepartitionForUnionAllBeforeHashJoin() anyTree( tableScan("nation", ImmutableMap.of("nationkey", "nationkey")))), exchange(REMOTE, REPARTITION, - project( - values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1")))))))) + exchange(LOCAL, REPARTITION, + project( + values(ImmutableList.of("expr"), ImmutableList.of(ImmutableList.of(new GenericLiteral("BIGINT", "1"))))))))) .right( anyTree( exchange(REMOTE, REPARTITION, - anyTree( - tableScan("region", ImmutableMap.of("regionkey", "regionkey"))))))))); + exchange(LOCAL, REPARTITION, + anyTree( + tableScan("region", ImmutableMap.of("regionkey", "regionkey")))))))))); } @Test @@ -229,15 +231,17 @@ public void testForcePartitioningMarkDistinctInput() node(MarkDistinctNode.class, anyTree( exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition1", "partition2"), - project( - values( - ImmutableList.of("field", "partition2", "partition1"), - ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1")))))), + exchange(LOCAL, REPARTITION, + 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"), - project( - values( - ImmutableList.of("partition3", "partition4", "field_0"), - ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1")))))))))); + exchange(LOCAL, REPARTITION, + project( + values( + ImmutableList.of("partition3", "partition4", "field_0"), + ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1"))))))))))); assertDistributedPlan( query, @@ -248,15 +252,17 @@ public void testForcePartitioningMarkDistinctInput() node(MarkDistinctNode.class, anyTree( exchange(REMOTE, REPARTITION, ImmutableList.of(), ImmutableSet.of("partition1"), - project( - values( - ImmutableList.of("field", "partition2", "partition1"), - ImmutableList.of(ImmutableList.of(new LongLiteral("1"), new LongLiteral("2"), new LongLiteral("1")))))), + exchange(LOCAL, REPARTITION, + 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"), - project( - values( - ImmutableList.of("partition3", "partition4", "field_0"), - ImmutableList.of(ImmutableList.of(new LongLiteral("3"), new LongLiteral("4"), new LongLiteral("1")))))))))); + exchange(LOCAL, REPARTITION, + 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 36e1e33dea18..ac910095e93a 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,7 +168,8 @@ public void testLocalScaledPartitionedWriterWithoutSupportForMultipleWritersPerP ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); } @Test(dataProvider = "taskScaleWritersOption") @@ -215,8 +216,9 @@ public void testLocalScaledPartitionedWriterWithoutSupportsForReportingWrittenBy project( exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); assertDistributedPlan( "INSERT INTO connector_partitioned_table SELECT * FROM source_table", @@ -232,7 +234,8 @@ public void testLocalScaledPartitionedWriterWithoutSupportsForReportingWrittenBy ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); } @DataProvider @@ -259,8 +262,9 @@ public void testLocalScaledPartitionedWriterForSystemPartitioning() project( exchange(LOCAL, REPARTITION, SCALED_WRITER_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); assertDistributedPlan( "INSERT INTO system_partitioned_table SELECT * FROM source_table", @@ -277,8 +281,9 @@ public void testLocalScaledPartitionedWriterForSystemPartitioning() project( exchange(LOCAL, REPARTITION, FIXED_HASH_DISTRIBUTION, exchange(REMOTE, REPARTITION, FIXED_HASH_DISTRIBUTION, - project( - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + project( + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))))); } @Test @@ -309,7 +314,8 @@ public void testLocalScaledPartitionedWriterForConnectorPartitioning() ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, scaledPartitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year")))))))); assertDistributedPlan( "INSERT INTO connector_partitioned_table SELECT * FROM source_table", @@ -325,6 +331,7 @@ public void testLocalScaledPartitionedWriterForConnectorPartitioning() ImmutableList.of("customer", "year"), exchange(LOCAL, REPARTITION, partitioningHandle, exchange(REMOTE, REPARTITION, partitioningHandle, - tableScan("source_table", ImmutableMap.of("customer", "customer", "year", "year"))))))); + exchange(LOCAL, REPARTITION, FIXED_ARBITRARY_DISTRIBUTION, + 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 2d934be5b4e8..4c7d84a918f2 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,21 +125,22 @@ public void testCoalesceWithManyArguments() exchange( REMOTE, 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"))))))))))); + 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")))))))))))); } @Test @@ -155,20 +156,21 @@ public void testComplexArgumentToCoalesce() exchange( REMOTE, 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"))))))))))); + 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")))))))))))); } } 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 d618a3d68f4a..d8a6a5af462e 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,7 +85,8 @@ public void testWindow() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus")))))))); + exchange(LOCAL, REPARTITION, + project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus"))))))))); assertDistributedPlan("SELECT row_number() OVER (PARTITION BY orderstatus) FROM orders", anyTree( @@ -93,7 +94,8 @@ public void testWindow() .partitionBy(ImmutableList.of("orderstatus")), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - project(tableScan("orders", ImmutableMap.of("orderstatus", "orderstatus")))))))); + exchange(LOCAL, REPARTITION, + 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( @@ -105,13 +107,14 @@ public void testWindow() .partial(false), exchange(LOCAL, GATHER, exchange(REMOTE, 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"))))))))); + 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")))))))))); } @Test @@ -175,16 +178,17 @@ public void testWindowAfterJoin() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, 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"))))))))))))); + 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")))))))))))))); } @Test @@ -199,7 +203,8 @@ public void testWindowAfterAggregation() project(aggregation(singleGroupingSet("custkey"), ImmutableMap.of(), Optional.empty(), FINAL, exchange(LOCAL, GATHER, project(exchange(REMOTE, REPARTITION, - anyTree(tableScan("orders", ImmutableMap.of("custkey", "custkey"))))))))))); + exchange(LOCAL, REPARTITION, + 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)", @@ -209,9 +214,11 @@ public void testWindowAfterAggregation() .addFunction(functionCall("rank", Optional.empty(), ImmutableList.of())), exchange(LOCAL, GATHER, exchange(REMOTE, REPARTITION, - 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")))))))))))); + 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")))))))))))))); } } 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 0fe2afe4d1be..76f5d27d33e9 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 @@ -451,13 +451,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(ProjectNode.class, node(TableScanNode.class)))))); + node(MarkDistinctNode.class, node(ExchangeNode.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(ProjectNode.class, node(TableScanNode.class)))))); + node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))))))); Session withoutMarkDistinct = Session.builder(getSession()) .setSystemProperty(USE_MARK_DISTINCT, "false") @@ -562,7 +562,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, ProjectNode.class); + .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.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 @@ -579,7 +579,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(ProjectNode.class, node(TableScanNode.class)))))); + node(MarkDistinctNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))))))); } } } @@ -973,7 +973,7 @@ public void testTopNPushdown() .isNotFullyPushedDown( node(TopNNode.class, // FINAL TopN anyTree(node(JoinNode.class, - node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class))), // no PARTIAL TopN + node(ExchangeNode.class, node(ExchangeNode.class, node(ProjectNode.class, node(TableScanNode.class)))), // no PARTIAL TopN anyTree(node(TableScanNode.class)))))); } @@ -1126,7 +1126,6 @@ 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 d4ac5e4d28ce..ba2d1e75228b 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 @@ -2152,7 +2152,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(1)); + assertLocalRepartitionedExchangesCount(2)); 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 d55f98c14b9a..2e26447df3bb 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,14 +170,16 @@ public void testPrunePartitionLikeFilter() .equiCriteria("L_STR_PART", "R_STR_COL") .left( exchange(REMOTE, REPARTITION, - project( - filter("\"like\"(L_STR_PART, \"$like_pattern\"('t%'))", - tableScan("table_str_partitioned", Map.of("L_INT_COL", "int_col", "L_STR_PART", "str_part")))))) + exchange(LOCAL, REPARTITION, + project( + filter("\"like\"(L_STR_PART, \"$like_pattern\"('t%'))", + tableScan("table_str_partitioned", Map.of("L_INT_COL", "int_col", "L_STR_PART", "str_part"))))))) .right(exchange(LOCAL, exchange(REMOTE, REPARTITION, - project( - filter("R_STR_COL IN ('three', CAST('two' AS varchar(5))) AND \"like\"(R_STR_COL, \"$like_pattern\"('t%'))", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); + exchange(LOCAL, REPARTITION, + project( + filter("R_STR_COL IN ('three', CAST('two' AS varchar(5))) AND \"like\"(R_STR_COL, \"$like_pattern\"('t%'))", + tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col"))))))))))); } @Test @@ -194,15 +196,17 @@ public void testSubsumePartitionFilter() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - project( - filter("true", // dynamic filter - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) + exchange(LOCAL, REPARTITION, + 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, - project( - filter("R_INT_COL IN (2, 3, 4)", - tableScan("table_unpartitioned", Map.of("R_STR_COL", "str_col", "R_INT_COL", "int_col")))))))))); + 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"))))))))))); } @Test @@ -220,15 +224,17 @@ public void testSubsumePartitionPartOfAFilter() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - project( - filter("L_STR_COL != 'three'", - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) + 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"))))))) .right( exchange(LOCAL, exchange(REMOTE, 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")))))))))); + 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"))))))))))); } @Test @@ -246,15 +252,17 @@ public void testSubsumePartitionPartWhenOtherFilterNotConvertibleToTupleDomain() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, 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")))))) + 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"))))))) .right( exchange(LOCAL, exchange(REMOTE, 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")))))))))); + 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"))))))))))); } @Test @@ -272,15 +280,17 @@ public void testSubsumePartitionFilterNotConvertibleToTupleDomain() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - project( - filter("L_INT_PART % 2 = 0", - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) + 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"))))))) .right( exchange(LOCAL, exchange(REMOTE, 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")))))))))); + 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"))))))))))); } @Test @@ -295,15 +305,17 @@ public void testFilterDerivedFromTableProperties() .equiCriteria("L_INT_PART", "R_INT_COL") .left( exchange(REMOTE, REPARTITION, - project( - filter("true", //dynamic filter - tableScan("table_int_partitioned", Map.of("L_INT_PART", "int_part", "L_STR_COL", "str_col")))))) + exchange(LOCAL, REPARTITION, + 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, - 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")))))))))); + 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"))))))))))); } @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 491d988def4d..eba30762e7b8 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, ProjectNode.class); + .isNotFullyPushedDown(MarkDistinctNode.class, ExchangeNode.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 62194425ed08..f14bb1c01a51 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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, ProjectNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.class, ExchangeNode.class, AggregationNode.class, MarkDistinctNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, ProjectNode.class, AggregationNode.class); + .isNotFullyPushedDown(AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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, AggregationNode.class, ProjectNode.class); + .isNotFullyPushedDown(ProjectNode.class, AggregationNode.class, ExchangeNode.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 f9c3d8b58a05..603d54f666d6 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,8 +56,9 @@ protected QueryRunner createQueryRunner() }) .setExtraProperties(FaultTolerantExecutionConnectorTestHelper.getExtraProperties()) // keep limits lower to test edge cases - .addExtraProperty("dynamic-filtering.small-partitioned.max-distinct-values-per-driver", "10") - .addExtraProperty("dynamic-filtering.small-broadcast.max-distinct-values-per-driver", "10") + .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") .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 61a20a28c4d3..4505e1d574d5 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,12 +10,13 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store @@ -24,16 +25,18 @@ local exchange (GATHER, SINGLE, []) final aggregation over (sr_store_sk_15) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk_15"]) - 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 (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 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 7605dbd6e9ad..522307dea3c5 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,39 +6,44 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_23"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_229"]) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_132) + 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 local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["d_week_seq_178"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 70aae3fa1bdd..4728fd4dc83f 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 99aa88843f21..adf59a0d1a7c 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,100 +3,118 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): 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, ["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 (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 local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_574"]) - 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"]) + 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, []) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_1634"]) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk_596"]) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - 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, ["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, []) + 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_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) + 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_1298"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_1610"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - 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"]) - 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) + 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_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 + 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, ["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"]) - 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) + 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 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 + 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_customer_sk_171"]) - scan customer + 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 + 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 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 f98eebbfad6e..a3e45e69889a 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,10 +45,12 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_96", "ws_order_number_110"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 d3d2cfba574a..78c06d9102ba 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,39 +10,45 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer 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, []) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - partial aggregation over (i_category_55) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 cbab34a925c5..c13e8aa2f577 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,19 +6,21 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 432e37f5b19c..b833e024e439 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,31 +6,36 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_39"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (ca_zip_31) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ca_zip_31) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_34"]) - 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 + 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 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 1cee500dbb4b..74c23aa40452 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,47 +3,53 @@ 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"]) - 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + 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"]) + 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 3e8cc6b9923a..9c95b298e45d 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,67 +2,79 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_85"]) - 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"]) + 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, []) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + local exchange (GATHER, SINGLE, []) + 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 - 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"]) + 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, []) scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk_600"]) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_sales + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - 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, []) + 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_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) + 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_375"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_576"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - 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 + 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 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 f7ec908dd084..78a75ae079e8 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,17 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 515bc8719e42..7a12976d2b07 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,22 +5,24 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + 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 ac9f82f11b05..c7254869e697 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,60 +9,66 @@ 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"]) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) join (INNER, REPLICATED): - scan store_sales + 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -90,60 +96,66 @@ 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"]) - partial aggregation over (i_brand_id_495, i_category_id_499, i_class_id_497) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_495, i_category_id_499, i_class_id_497) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -174,57 +186,63 @@ 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"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk_1097"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - partial aggregation over (i_brand_id_1220, i_category_id_1224, i_class_id_1222) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1220, i_category_id_1224, i_class_id_1222) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item remote exchange (REPARTITION, HASH, ["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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1314, i_category_id_1318, i_class_id_1316) 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 item + scan item remote exchange (REPARTITION, HASH, ["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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1408, i_category_id_1412, i_class_id_1410) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 946683bd17b5..a9f23b963ee8 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,16 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 2583381afcd8..ec35505379b1 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,32 +4,35 @@ final aggregation over () partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_25"]) - 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): + 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 (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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 add2e9b8dfad..402e64a4b7b7 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,34 +3,37 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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"]) - 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 + 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 f3ff54aad741..79fba25ad0a6 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,34 +4,37 @@ 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) - 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_item_sk"]) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) + remote exchange (REPARTITION, HASH, ["cs_bill_customer_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_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + 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"]) join (INNER, PARTITIONED): - 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 + remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_3"]) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + 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 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 43f0630f3559..98f1c759ed77 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,25 +3,26 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 95a4e8334e25..b2605bcac742 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,17 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 c127db88db46..24265297745e 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,17 +3,18 @@ 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"]) - partial aggregation over (i_item_id, w_warehouse_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id, w_warehouse_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan inventory + 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 date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 81cbaf214536..d15fdcf05bf2 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,31 +4,35 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_9, ss_item_sk, substr$gid) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -36,10 +40,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_47) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_47"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -51,42 +57,48 @@ final aggregation over () partial aggregation over (ss_customer_sk_78) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_78"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_103"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_199) + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -94,10 +106,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_292) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_292"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_317"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -109,11 +123,13 @@ final aggregation over () partial aggregation over (ss_customer_sk_343) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_343"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_368"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 b24eafd9738a..e039646ac696 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,35 +5,40 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -51,20 +56,24 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_125"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 06e9bc127b43..48b7cc7657a3 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,35 +3,40 @@ 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"]) - 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 (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 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 (REPARTITION, HASH, ["s_store_sk"]) - scan store + 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 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 10e8a2551020..959a8e6cf638 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,19 +6,21 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): 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 + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e15f77571f95..a743f4906e3f 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,19 +6,21 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 d917765996a2..747a5c351502 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,39 +9,45 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e563e2d9b18a..0b0b1066f9fd 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,34 +3,37 @@ 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"]) - 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 + 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 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 120f70ae6d63..f966fccd867d 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,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 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"]) - 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 (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 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 2ae87cf0822b..442731513c1f 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,30 +10,35 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_64"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_149", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - 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 + 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 join (INNER, PARTITIONED): final aggregation over (ca_county_293, d_qoy_265, d_year_261) local exchange (GATHER, SINGLE, []) @@ -41,58 +46,68 @@ 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_223"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_382", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - final aggregation over (ca_county_382, d_qoy_354, d_year_350) + 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) 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) + 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, ["ca_address_sk_375"]) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk_312"]) + remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_sales + scan store_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 + 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 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 3fe019ccd451..0063a2a6a620 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,23 +7,25 @@ final aggregation over () final aggregation over (cs_item_sk_20) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_20"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 fb7279d7b9b0..c544e95a3541 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,26 +9,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_manufact_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - 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 + 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 partial aggregation over (i_manufact_id_103) final aggregation over (i_manufact_id_103) local exchange (GATHER, SINGLE, []) @@ -36,26 +39,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_manufact_id_103) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) 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_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_90"]) - 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 + 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 partial aggregation over (i_manufact_id_210) final aggregation over (i_manufact_id_210) local exchange (GATHER, SINGLE, []) @@ -65,22 +71,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) 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 (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_181"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_197"]) - 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 + 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 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 64d4ce451141..42dd07b77e7e 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,17 +8,18 @@ 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"]) - partial aggregation over (ss_customer_sk, ss_ticket_number) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_customer_sk, ss_ticket_number) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + 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 d88f27b4c7b6..362715d1d4c4 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,41 +12,47 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address 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 (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, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 (REPLICATE, BROADCAST, []) 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 (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 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 630dacce4c1e..5221892abb85 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,15 +8,17 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 3ae83aae01f0..008288ede797 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,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + 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 + 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 94d514729231..399554ef1411 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,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_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 (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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 (REPARTITION, ROUND_ROBIN, []) + 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 0761f10365a3..a96d0b7e0a46 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,39 +3,43 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_11", "inv_warehouse_sk_12"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + 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 b7cf1dd0bae2..85412806878e 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,10 +9,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 58a96c8d1ec2..c67211745712 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,10 +3,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - 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 + 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 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 17cc12f10e9e..d6a3c8cac82b 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_category, i_category_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_category, i_category_id) join (INNER, REPLICATED): - scan store_sales + 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 item + 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 9d9e36441a12..53c0550a78af 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,17 +3,19 @@ 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"]) - 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 + 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 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 bf870eae841a..bc31761eebff 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,42 +4,48 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - 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 (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk_29) + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) - partial aggregation over (ss_item_sk_29) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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_63) + final aggregation over (ss_store_sk_6) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_63"]) - partial aggregation over (ss_store_sk_63) - scan store_sales + 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 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 7dd52a5c0765..474216977dca 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,19 +8,22 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 (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 cd659271967f..6a7aabefa820 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,31 +2,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_16"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - 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): + 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 store_sales + 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 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 household_demographics + 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 0911429ef31b..ce0faaaab7ba 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,58 +4,64 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + 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 d002184141c7..3f5a3466a293 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,18 +5,20 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 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/hive/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/partitioned/q49.plan.txt index 6b7ece910520..eb091b77b5f5 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,53 +3,59 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_74", "expr_78", "rank_76", "rank_77"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["expr_128", "expr_133", "rank_130", "rank_131", "sr_item_sk"]) - 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 + 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 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 9f5732a36de7..89f63e55a780 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,21 +3,22 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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, []) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 572cebe3eb35..0740bb6b16e6 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,26 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - 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 + 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 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 33752e693e6c..ab4fb497ba27 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + 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 8b8179689d06..8fec7b4a24ec 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,21 +2,24 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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, 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 + 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 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 a7911d69a280..2572b1c3d263 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,64 +3,71 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - 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 (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 local exchange (GATHER, SINGLE, []) - 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): + 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) join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales + scan customer 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, []) + 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 (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) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_116"]) - partial aggregation over (expr_116) - scan date_dim + 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 + 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 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 0322599f01e5..ad84f78b05f8 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,13 +3,14 @@ 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"]) - partial aggregation over (i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + 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 0f2f93f9a976..d1026992d606 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,26 +9,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address partial aggregation over (i_item_id_91) final aggregation over (i_item_id_91) local exchange (GATHER, SINGLE, []) @@ -36,26 +39,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_91) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_74"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address partial aggregation over (i_item_id_198) final aggregation over (i_item_id_198) local exchange (GATHER, SINGLE, []) @@ -63,23 +69,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_198) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_181"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 4bed99d3b879..7cab3797a1b1 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,58 +4,64 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_109", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_245", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 fb65ed72a690..8a6e79c33754 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,80 +8,88 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_75) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_74"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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, []) - remote exchange (REPLICATE, BROADCAST, []) + 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 (GATHER, SINGLE, []) - scan date_dim + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): - scan store_sales + 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 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 + 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 719af5ccc77b..e919b701b5bb 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,47 +2,53 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["d_week_seq", "s_store_id"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - final aggregation over (d_week_seq, ss_store_sk) + 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 local exchange (GATHER, SINGLE, []) - 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 + 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 + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_211", "s_store_id_124"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - final aggregation over (d_week_seq_80, ss_store_sk_55) + 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 local exchange (GATHER, SINGLE, []) - 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 + 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 + 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 c573da2e1725..7b0d31a2a635 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,26 +9,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_8) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - 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 (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 partial aggregation over (i_item_id_91) final aggregation over (i_item_id_91) local exchange (GATHER, SINGLE, []) @@ -36,26 +39,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_91) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_74"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address partial aggregation over (i_item_id_198) final aggregation over (i_item_id_198) local exchange (GATHER, SINGLE, []) @@ -63,23 +69,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_198) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_181"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 958cb1b32e1b..28da7501da8e 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,53 +5,57 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk_112"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 61172c74fc4f..a00a28581512 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,21 +3,22 @@ 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"]) - partial aggregation over (sm_type, substr$gid, web_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sm_type, substr$gid, web_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 23a54b451e8f..eeebbe56a704 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,21 +2,24 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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, 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 + 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 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 0ab88e33c808..2907b89404be 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,177 +3,195 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_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): + 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"]) join (INNER, REPLICATED): join (INNER, REPLICATED): 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, 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, ["ss_item_sk", "ss_ticket_number"]) - join (INNER, REPLICATED): - scan store_sales + 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 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"]) - join (INNER, REPLICATED): - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) + 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 (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 + 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, ["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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 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 + 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 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_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, 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"]) join (INNER, REPLICATED): join (INNER, REPLICATED): 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, 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, ["ss_item_sk_154", "ss_ticket_number_161"]) - join (INNER, REPLICATED): - scan store_sales + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - 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) + 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 (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 + 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 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 + 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 744c84a2143a..eaf127c7f5a0 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,30 +5,34 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - 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 (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 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 785f1198b31c..2abe02a49224 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,51 +6,55 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 6c71f017bae6..0f1bf39938ff 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,31 +2,34 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk_17"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - 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): + 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 store_sales + 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 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 household_demographics + 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 18406802cc1b..2fdcca1cc11c 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,43 +9,49 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 (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, ["ss_customer_sk"]) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales + 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 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 + 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"]) - 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 (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"]) - 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 (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 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 7f94b5469eb9..d6d865362e78 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,26 +8,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_98) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (s_state_57) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (s_state_57) join (INNER, REPLICATED): - scan store_sales + 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 store + 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 02c955d77dfd..c5dc8d44d272 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,4 +29,5 @@ remote exchange (GATHER, SINGLE, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["t_time_sk"]) - scan time_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 8c52b501aa32..17448606409b 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,46 +3,50 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan date_dim 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + 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 8ca487c16558..ab6eee0fb93b 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,17 +8,18 @@ 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"]) - partial aggregation over (ss_customer_sk, ss_ticket_number) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_customer_sk, ss_ticket_number) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 13345b1ee1f0..294b48c00816 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,67 +2,79 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_75"]) - 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"]) + 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, []) 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 (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["c_customer_sk_74"]) + local exchange (REPARTITION, ROUND_ROBIN, []) 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"]) - 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 (REPARTITION, ROUND_ROBIN, []) + 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_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) + 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_346"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_533"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - 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 + 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 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 20311d91862a..e89787932b7a 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,101 +5,119 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_34", "i_category_id_38", "i_class_id_36", "i_manufact_id_40"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_100", "i_category_id_104", "i_class_id_102", "i_manufact_id_106"]) - 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 + 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 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_332", "i_category_id_336", "i_class_id_334", "i_manufact_id_338"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_452", "i_category_id_456", "i_class_id_454", "i_manufact_id_458"]) - 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 + 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 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 d7b6d9db21ac..393fbfd02336 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,36 +3,40 @@ 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"]) - partial aggregation over (d_qoy_11, d_year_10, expr_144, expr_145, i_category_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_11, d_year_10, expr_144, expr_145, i_category_6) join (INNER, REPLICATED): - scan store_sales + 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 item + 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"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_120, d_year_116, expr_141, expr_143, i_category_97) 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 item + 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 f5f8b7b62eea..a5c406789d66 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,74 +9,80 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - partial aggregation over (ss_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - partial aggregation over (sr_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sr_store_sk) join (INNER, REPLICATED): - scan store_returns + 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 + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - 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 (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 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"]) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - partial aggregation over (ws_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - partial aggregation over (wr_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): - scan web_returns + 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, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + 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 c21aa577ba08..190bea4850cf 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,10 +8,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -25,10 +27,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -39,10 +43,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 5efde1d3d8c4..7ae16b239b0c 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,23 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 a4b2da9fdab0..cbfdaea7dfa3 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,10 +15,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -41,10 +43,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -67,10 +71,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 ce0e336b201c..cc48bf7e28db 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,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 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 ab193ba29af9..358e01cc84f0 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,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + 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 + 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 87e080b8e1e2..2e6d37fdd4e8 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,87 +8,99 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_80) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_79"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk"]) - 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 + 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 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 2ae4cae3b89b..09809dcef670 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,36 +3,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - partial aggregation over (r_reason_desc) - join (INNER, REPLICATED): - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): + 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, []) join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + 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 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"]) - 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 + remote exchange (REPLICATE, BROADCAST, []) + scan web_page local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_page - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan reason + 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 990089879464..f880adf8bf2d 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,13 +5,14 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 94d514729231..399554ef1411 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,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_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 (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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 (REPARTITION, ROUND_ROBIN, []) + 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 2737c0093fbb..4446fe5ca255 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,20 +2,22 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 c113b8bda923..65d6cda93e78 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,21 +11,24 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 (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 4d76e724d197..7c55cbd98e00 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,23 +7,25 @@ final aggregation over () final aggregation over (ws_item_sk_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_8"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 ca9dc7543db7..a97d8ef18468 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,10 +7,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 259bf30db48e..66f3ade77f16 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,24 +8,27 @@ 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - partial aggregation over (wr_order_number) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 ea818962db9f..6b72edeb5d9f 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,42 +4,48 @@ final aggregation over () partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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): + 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) join (INNER, PARTITIONED, can skip output duplicates): - remote exchange (REPARTITION, HASH, ["ws_order_number_109"]) - scan web_sales + 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 local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["ws_order_number_147"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 7b9c83293534..24bc0bdee176 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,18 +6,20 @@ 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"]) - 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 + 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 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"]) - 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 + 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 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 a6a04beb1e8b..aa5503f47d8e 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,17 +3,20 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 6e2c1451c321..e17b712a1894 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,21 +3,22 @@ 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"]) - partial aggregation over (cc_name, sm_type, substr$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cc_name, sm_type, substr$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 da4afcb7be2b..8dc6946909e2 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,37 +4,42 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - 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 (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 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 ded78380a2fc..db79dd1da69e 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,39 +6,44 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_22"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_226"]) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_129) + 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 local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["d_week_seq_175"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 70aae3fa1bdd..4728fd4dc83f 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 420fa6a923ec..2b1977a14f13 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,99 +4,117 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_877"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_1619"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1594"]) - scan customer + 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 local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_568"]) - 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_1310"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_1285"]) - scan customer + 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"]) - 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"]) - 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 (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) 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_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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_customer_sk_193"]) + 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"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_169"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 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 4155bb638c70..9ab2adedbf44 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,10 +45,12 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_92", "ws_order_number_106"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 2a72afdda3fa..52553713d3ad 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,29 +9,33 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -40,8 +44,9 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_55) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_55"]) - partial aggregation over (i_category_55) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 32f4763b443d..d209bd09ae61 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,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): 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 date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 432e37f5b19c..b833e024e439 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,31 +6,36 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_39"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (ca_zip_31) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ca_zip_31) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_34"]) - 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 + 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 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 7d60d2a5fdd5..a53dd7834678 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,48 +3,54 @@ 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"]) - 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - 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, ["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, []) + 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"]) - 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 + 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) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + 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 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 3edd5dbb25a4..b71d7c228bdd 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,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_84"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_396"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_371"]) - scan customer + 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 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, ["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 (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_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) + 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, ["ws_bill_customer_sk_595"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_570"]) - scan customer + 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 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 6f94c98c5224..14d23e717ece 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,16 +2,18 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 fe1028ad939b..5e73c397329e 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,60 +9,66 @@ 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"]) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) join (INNER, REPLICATED): - scan store_sales + 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -90,60 +96,66 @@ 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"]) - partial aggregation over (i_brand_id_488, i_category_id_492, i_class_id_490) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_488, i_category_id_492, i_class_id_490) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -171,60 +183,66 @@ 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"]) - partial aggregation over (i_brand_id_1090, i_category_id_1094, i_class_id_1092) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1090, i_category_id_1094, i_class_id_1092) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 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 f359fbefb40f..4a9d57b8e143 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,16 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 ac81595cda39..514a586ee0de 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,28 +7,31 @@ final aggregation over () 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 + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_26"]) - 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): + 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, []) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 47c73f484403..326846b8f36d 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,32 +6,36 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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"]) - 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 + 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 f338fb880363..4e39a11a0e9a 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,13 +20,16 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk_3"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 5230f46699ba..c3cd8014d9b4 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,24 +6,27 @@ 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 852a9d2a3e5c..2dce6eadbb48 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,16 +2,18 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 50e40917c430..e8067ff5be06 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,17 +3,18 @@ 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"]) - partial aggregation over (i_item_id, w_warehouse_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id, w_warehouse_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan inventory + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 56d143391099..ae9b7d2ec96f 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,31 +4,35 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_8, ss_item_sk, substr$gid) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -36,10 +40,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_47) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_47"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -51,42 +57,48 @@ final aggregation over () partial aggregation over (ss_customer_sk_77) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_77"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_100"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_196) + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -94,10 +106,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_288) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_288"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_311"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -109,11 +123,13 @@ final aggregation over () partial aggregation over (ss_customer_sk_338) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_338"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_361"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 5a1ffc003e35..3147f15c942c 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,34 +5,39 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -50,20 +55,24 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_122"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 a21872e9dd27..8eeff963a4b4 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,32 +6,36 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 555bb67d3baa..dc978ab9fbae 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,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 ed0e2e73e9f3..22ec76d44200 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,19 +6,21 @@ 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"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) 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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 9bf8b7c08e32..df987db25c6b 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,39 +9,45 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 a21872e9dd27..8eeff963a4b4 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,32 +6,36 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 c56b520204f0..111401ec279f 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,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 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 efa718f41fb7..d1d09df16887 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,30 +10,35 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_63"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_147", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - 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 + 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 join (INNER, PARTITIONED): final aggregation over (ca_county_289, d_qoy_261, d_year_257) local exchange (GATHER, SINGLE, []) @@ -41,58 +46,68 @@ 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"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_282"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_377", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - final aggregation over (ca_county_377, d_qoy_349, d_year_345) + 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_377", "d_qoy_349", "d_year_345"]) - partial aggregation over (ca_county_377, d_qoy_349, d_year_345) + 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, ["ws_bill_addr_sk_309"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_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 + 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) + 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"]) + 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_194"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 fbb263a8d062..b214e66bcc58 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,23 +7,25 @@ final aggregation over () final aggregation over (cs_item_sk_21) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_21"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 6d9187d8b05b..0ef86c7948f0 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,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_8"]) - partial aggregation over (i_manufact_id_8) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_8) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_102) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_102) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_208) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_208) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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 8b6f823e1fe3..861d0f0bd0d8 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,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + 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 f1fed4317214..b7f586b43c2d 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,37 +13,43 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address 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 (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, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + scan catalog_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 3fe86fe51161..eb941375d1fb 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,17 +5,18 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 store + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 2582d9fa4122..1cad16277680 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,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 6590f3356d80..fb7e4491f3bb 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,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_46"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_101"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 c06e49455217..42f59e4472a9 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,39 +3,43 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_12", "inv_warehouse_sk_13"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + 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 266142fc4c1a..62288a2c3dc5 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,10 +9,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 58a96c8d1ec2..c67211745712 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,10 +3,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - 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 + 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 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 87792b40cd98..aaf625b30700 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_category, i_category_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_category, i_category_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 9d9e36441a12..53c0550a78af 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,17 +3,19 @@ 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"]) - 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 + 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 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 093a41b0ae05..10edcc19a124 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,42 +4,48 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - 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 (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk_29) + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_29"]) - partial aggregation over (ss_item_sk_29) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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_62) + final aggregation over (ss_store_sk_7) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_62"]) - partial aggregation over (ss_store_sk_62) - scan store_sales + 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 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 6af76139314e..3104679c2c3b 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,19 +8,22 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 87414d60d6a0..cda34047522b 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,32 +2,37 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + scan household_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 (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_16"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 626ebbc4e55f..7c9fbe88943e 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,58 +4,64 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + 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 fed42e3537db..7afb18291ac0 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,56 +3,65 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_71", "expr_75", "rank_73", "rank_74"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["expr_123", "expr_128", "rank_125", "rank_126", "sr_item_sk"]) - 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 + 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 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 c2a9d481bf10..c28e1f11bc9a 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,14 +8,16 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales 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 (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 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 5020827b6750..bed68d8668d2 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,26 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - 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 + 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 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 70aae3fa1bdd..4728fd4dc83f 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,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 cd10ab3698e6..2206a00753f9 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,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 90dc47e1a40d..1e79f1eef876 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,64 +3,71 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - 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 (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 local exchange (GATHER, SINGLE, []) - 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): + 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) join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales + scan customer 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, []) + 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 (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) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_114"]) - partial aggregation over (expr_114) - scan date_dim + 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 + 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 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 f89f2f078b63..92c0bba58211 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,13 +3,14 @@ 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"]) - partial aggregation over (i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 be881ce912e9..9cda6612750d 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,28 +6,30 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_8"]) - partial aggregation over (i_item_id_8) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_8) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + 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 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 (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address partial aggregation over (i_item_id_90) final aggregation over (i_item_id_90) local exchange (GATHER, SINGLE, []) @@ -35,26 +37,29 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_90) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_73"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address partial aggregation over (i_item_id_196) final aggregation over (i_item_id_196) local exchange (GATHER, SINGLE, []) @@ -62,23 +67,26 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_196) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_addr_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_179"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e9f0b1bd10c0..23e9d7279988 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,58 +4,64 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_108", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_243", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 b5044356bbce..2aa7dc85d1da 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,30 +4,32 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): - scan store_sales + 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 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 + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) @@ -35,53 +37,59 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_73"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_200"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 8c0032f32687..b652ec89d3db 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,46 +4,53 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_210", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_56"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - final aggregation over (d_week_seq_79, ss_store_sk_56) + 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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_123"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 b42dce272fe4..fe3367ade51d 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,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_8"]) - partial aggregation over (i_item_id_8) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_8) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_90) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_90) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_196) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_196) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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 aa366b079da7..a466d09e6f9c 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,26 +6,28 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store 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 (REPARTITION, ROUND_ROBIN, []) + 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 @@ -35,23 +37,25 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_21"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_107"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 61172c74fc4f..a00a28581512 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,21 +3,22 @@ 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"]) - partial aggregation over (sm_type, substr$gid, web_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sm_type, substr$gid, web_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 23e28eddc4b7..1732f85548b8 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,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 6497c0e5dec4..e88e3b4f2ce0 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,177 +3,195 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - 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): + 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, []) scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + 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, []) 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, 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, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - join (INNER, REPLICATED): - scan store_sales + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics 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 (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store 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 + remote exchange (REPARTITION, HASH, ["c_customer_sk"]) + 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 (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["cd_demo_sk_84"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_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 promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band + 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_368", "s_zip_388", "ss_item_sk_152"]) - 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): + 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, []) scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + 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, []) 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, 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, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["ss_item_sk_152", "ss_ticket_number_159"]) - join (INNER, REPLICATED): - scan store_sales + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["sr_item_sk_178", "sr_ticket_number_185"]) - scan store_returns + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics 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"]) - 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 + scan store 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 + remote exchange (REPARTITION, HASH, ["c_customer_sk_395"]) + 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 (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["cd_demo_sk_428"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_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 promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band + 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/unpartitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/hive/unpartitioned/q65.plan.txt index 16d4126e5149..197f7ce1d450 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,30 +5,34 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - 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 (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 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 d78688a29a6c..5796c6a13754 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,51 +6,55 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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 ecbd5a004789..b6745add56f9 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,30 +4,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_17"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 d304792469c0..bc516ac6a348 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,48 +3,55 @@ 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"]) - 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"]) - 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 (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 (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 (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 + 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 (REPLICATE, BROADCAST, []) - scan date_dim + 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) + 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 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 ad464d39cb92..cd2a781d1350 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,26 +8,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_97) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (s_state_56) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (s_state_56) join (INNER, REPLICATED): - scan store_sales + 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 store + 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 9da6042a0c9d..4417571368bb 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,34 +9,37 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan customer_demographics 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 - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -45,4 +48,5 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 30d700ed0193..9fa35d4ee8c7 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,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + 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 213bb1464a96..85b74bb40480 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,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_74"]) - 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 (REPARTITION, ROUND_ROBIN, []) + 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_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) + 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, ["ws_bill_customer_sk_367"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_342"]) - scan customer + 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 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_552"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_527"]) - scan customer + 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 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 d5a331127722..904e76f040c1 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,101 +5,119 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_32", "i_category_id_36", "i_class_id_34", "i_manufact_id_38"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_96", "i_category_id_100", "i_class_id_98", "i_manufact_id_102"]) - 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 + 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 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_324", "i_category_id_328", "i_class_id_326", "i_manufact_id_330"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_442", "i_category_id_446", "i_class_id_444", "i_manufact_id_448"]) - 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 + 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 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 2c646494d11f..a946c27d3253 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,37 +3,42 @@ 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"]) - partial aggregation over (d_qoy_11, d_year_10, expr_142, expr_143, i_category_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_11, d_year_10, expr_142, expr_143, i_category_6) join (INNER, REPLICATED): - scan store_sales + 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 item + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_45"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim remote exchange (REPARTITION, HASH, ["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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_118, d_year_114, expr_139, expr_141, i_category_95) 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 item + 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 f5f8b7b62eea..a5c406789d66 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,74 +9,80 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - partial aggregation over (ss_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - partial aggregation over (sr_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sr_store_sk) join (INNER, REPLICATED): - scan store_returns + 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 + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - 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 (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 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"]) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - partial aggregation over (ws_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - partial aggregation over (wr_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): - scan web_returns + 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, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + 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 8644def442f8..0353497d6cec 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,48 +2,56 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 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, []) - 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 + 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 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 date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - 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 (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 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 date_dim + 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 5efde1d3d8c4..7ae16b239b0c 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,23 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 a4b2da9fdab0..cbfdaea7dfa3 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,10 +15,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -41,10 +43,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -67,10 +71,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 0f6856f9c813..0221aac3009a 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,14 +9,16 @@ local exchange (GATHER, SINGLE, []) 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 (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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -29,21 +31,24 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state_94) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_state_94"]) - 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 (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 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 532952036fc3..4232c710d2d6 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,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 cc4b500d368e..fbc09e235660 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,30 +7,34 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_79) local exchange (GATHER, SINGLE, []) @@ -38,57 +42,65 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_79) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_78"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk_210"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 ab3b2c159101..3c7c4fb1c2f8 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,36 +3,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - partial aggregation over (r_reason_desc) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (r_reason_desc) join (INNER, REPLICATED): - scan customer_demographics + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 990089879464..f880adf8bf2d 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,13 +5,14 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 6590f3356d80..fb7e4491f3bb 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,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_46"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_101"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 396d705bbc38..b5495ae3801b 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,20 +2,22 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + 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 f1c8dfe5de34..3427cfc56e6c 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,23 +9,26 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 2e92e9b3a1a9..f7f467b476d8 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,23 +7,25 @@ final aggregation over () final aggregation over (ws_item_sk_9) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_9"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 ca9dc7543db7..a97d8ef18468 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,10 +7,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 217dddd8d551..e643c7b023cc 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,28 +7,31 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - partial aggregation over (wr_order_number) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_26"]) - 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): + 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, []) join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 9daf91528a0a..956e3f99badd 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,35 +9,41 @@ 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_144"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_63"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_site + 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 7b9c83293534..24bc0bdee176 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,18 +6,20 @@ 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"]) - 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 + 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 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"]) - 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 + 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 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 9775b6c511f5..e5840c437809 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,16 +3,18 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 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 6e2c1451c321..e17b712a1894 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,21 +3,22 @@ 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"]) - partial aggregation over (cc_name, sm_type, substr$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cc_name, sm_type, substr$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q01.plan.txt index d091b5f71d8b..630021f6984d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q01.plan.txt @@ -4,37 +4,42 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q02.plan.txt index f26bc3efa106..a9e4ac5bcfad 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q02.plan.txt @@ -6,39 +6,44 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) + 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 local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q03.plan.txt index 70aae3fa1bdd..4728fd4dc83f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q03.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q04.plan.txt index 84c91be8eac4..954de084f516 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q04.plan.txt @@ -4,99 +4,117 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_1567"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - scan customer + 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 local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_1267"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_1244"]) - scan customer + 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"]) - 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"]) - 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 (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) 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_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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_customer_sk_183"]) + 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"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q05.plan.txt index 169f66124bda..da6352ecc72f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q05.plan.txt @@ -45,10 +45,12 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_81", "ws_order_number_95"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q06.plan.txt index fb3592d65b6e..1c7ee06ff527 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q06.plan.txt @@ -11,27 +11,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -40,8 +44,9 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - partial aggregation over (i_category_50) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q07.plan.txt index 32f4763b443d..d209bd09ae61 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q07.plan.txt @@ -3,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): 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 date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q08.plan.txt index 45b29afc325b..998b0b959fcb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q08.plan.txt @@ -6,31 +6,36 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (ca_zip_26) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q10.plan.txt index 7d60d2a5fdd5..a53dd7834678 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q10.plan.txt @@ -3,48 +3,54 @@ 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"]) - 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - 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, ["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, []) + 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"]) - 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 + 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) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q11.plan.txt index 668a1795f1c6..6fee2f7f3db5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q11.plan.txt @@ -3,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_380"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - scan customer + 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 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, ["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 (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_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) + 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, ["ws_bill_customer_sk_573"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_550"]) - scan customer + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q12.plan.txt index eb114fe9092d..609e3537018c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q12.plan.txt @@ -2,17 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q13.plan.txt index a4d7d7d75438..30470310dca0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q13.plan.txt @@ -8,20 +8,23 @@ final aggregation over () join (INNER, REPLICATED): 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 (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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q14.plan.txt index c8326b25e125..21eb6df883d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q14.plan.txt @@ -9,60 +9,66 @@ 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"]) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) join (INNER, REPLICATED): - scan store_sales + 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -90,60 +96,66 @@ 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"]) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -171,60 +183,66 @@ 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"]) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q15.plan.txt index 56302f05c739..b22f9d12537c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q15.plan.txt @@ -6,16 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q16.plan.txt index 2c79deb552b5..67c9ba77fd3d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q16.plan.txt @@ -7,28 +7,31 @@ final aggregation over () 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 + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_22"]) - 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): + 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, []) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q17.plan.txt index 49c98c0b8dda..7a9af86c0c6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q17.plan.txt @@ -6,32 +6,36 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q18.plan.txt index 5f2dfaaeb4fa..7f9cbb3ebf63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q18.plan.txt @@ -4,33 +4,35 @@ 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) - 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): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_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 customer_demographics + 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"]) + 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"]) join (INNER, PARTITIONED): - 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 + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q19.plan.txt index c500acf6fca3..967aa874de69 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q19.plan.txt @@ -7,23 +7,26 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 item + 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/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q20.plan.txt index 852a9d2a3e5c..2dce6eadbb48 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q20.plan.txt @@ -2,16 +2,18 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q21.plan.txt index 50e40917c430..e8067ff5be06 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q21.plan.txt @@ -3,17 +3,18 @@ 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"]) - partial aggregation over (i_item_id, w_warehouse_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id, w_warehouse_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan inventory + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q23.plan.txt index a98a247c3340..64f6afa02051 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q23.plan.txt @@ -4,31 +4,35 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -36,10 +40,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_42) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_42"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -51,42 +57,48 @@ final aggregation over () partial aggregation over (ss_customer_sk_70) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_70"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_93"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_184) + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -94,10 +106,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_273) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_273"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -109,11 +123,13 @@ final aggregation over () partial aggregation over (ss_customer_sk_321) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_321"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_344"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q24.plan.txt index 243f543065be..791a3935f4a9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q24.plan.txt @@ -5,35 +5,40 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -52,19 +57,23 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_12", "ss_ticket_number_19"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_37", "sr_ticket_number_44"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q25.plan.txt index 8eedb8e05888..4dcfe955597d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q25.plan.txt @@ -7,31 +7,35 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + 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/partitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q26.plan.txt index 555bb67d3baa..dc978ab9fbae 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q26.plan.txt @@ -3,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q27.plan.txt index b637b1d27f97..22ec76d44200 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/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) - 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, 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 - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q28.plan.txt index 069ba6e93218..f64d5530bbf2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q28.plan.txt @@ -9,39 +9,45 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q29.plan.txt index 8eedb8e05888..4dcfe955597d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q29.plan.txt @@ -7,31 +7,35 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + 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/partitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q30.plan.txt index 4db8148f3ed6..f2d3b05fe566 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q30.plan.txt @@ -4,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q31.plan.txt index 04a7a0fc2cd4..c65ae2ca3c5a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q31.plan.txt @@ -10,30 +10,35 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_59"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - 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 + 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 join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -41,58 +46,68 @@ 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"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_269"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) + 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_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) + 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, ["ws_bill_addr_sk_294"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_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 + 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) + 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"]) + 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_184"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q32.plan.txt index e09ec6de273c..be2a74f9a07a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q32.plan.txt @@ -7,23 +7,25 @@ final aggregation over () final aggregation over (cs_item_sk_18) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_18"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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/partitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q33.plan.txt index 1b2427d4e654..2502d90464f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q33.plan.txt @@ -6,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - partial aggregation over (i_manufact_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_95) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_95) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_196) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_196) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q34.plan.txt index 8b6f823e1fe3..861d0f0bd0d8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q34.plan.txt @@ -3,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q35.plan.txt index f94ec8be8411..8353da6955f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q35.plan.txt @@ -11,39 +11,45 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer 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 (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, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + scan catalog_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/partitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q36.plan.txt index d640f5ac0d14..c7aa800fe3c9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q36.plan.txt @@ -5,17 +5,18 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 store + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q37.plan.txt index 2582d9fa4122..1cad16277680 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q37.plan.txt @@ -3,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q38.plan.txt index 9f02b3c87024..8d422b83cadf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q38.plan.txt @@ -11,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q39.plan.txt index eb2349ef1574..b74a3c77c64a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q39.plan.txt @@ -3,39 +3,43 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_8", "inv_warehouse_sk_9"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q40.plan.txt index 266142fc4c1a..62288a2c3dc5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q40.plan.txt @@ -9,10 +9,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q41.plan.txt index 58a96c8d1ec2..c67211745712 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q41.plan.txt @@ -3,10 +3,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q42.plan.txt index 17cc12f10e9e..d6a3c8cac82b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q42.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_category, i_category_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_category, i_category_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q43.plan.txt index 9d9e36441a12..53c0550a78af 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q43.plan.txt @@ -3,17 +3,19 @@ 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"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q44.plan.txt index 7cd4ca5a0807..f8aeb77ec755 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q44.plan.txt @@ -4,42 +4,48 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - 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 (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk_27) + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_27"]) - partial aggregation over (ss_item_sk_27) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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_59) + final aggregation over (ss_store_sk_6) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_59"]) - partial aggregation over (ss_store_sk_59) - scan store_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q45.plan.txt index 5b30cdf7426c..b275b76606e4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q45.plan.txt @@ -10,20 +10,23 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q46.plan.txt index e8aa3db7886c..6ddeddd43120 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q46.plan.txt @@ -4,30 +4,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q47.plan.txt index 2d63bde52f55..93faff459e9c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q47.plan.txt @@ -4,58 +4,64 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q48.plan.txt index 7eb552bd09e3..4e0866c981c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q48.plan.txt @@ -7,17 +7,20 @@ final aggregation over () remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) 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 (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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q49.plan.txt index 6f687392953d..66381314bec4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q49.plan.txt @@ -3,56 +3,65 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q50.plan.txt index c2a9d481bf10..c28e1f11bc9a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q50.plan.txt @@ -8,14 +8,16 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales 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 (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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q51.plan.txt index ed5e8607b930..06b2887fa30d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q51.plan.txt @@ -2,26 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q52.plan.txt index 33752e693e6c..ab4fb497ba27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q52.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q53.plan.txt index cd10ab3698e6..2206a00753f9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q53.plan.txt @@ -2,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q54.plan.txt index 23ee4dacf66b..73276be84ef1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q54.plan.txt @@ -3,64 +3,71 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - 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 (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 local exchange (GATHER, SINGLE, []) - 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): + 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) join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales + scan customer 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, []) + 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 (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) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - partial aggregation over (expr_104) - scan date_dim + 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q55.plan.txt index 0322599f01e5..ad84f78b05f8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q55.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q56.plan.txt index e96c8c31aabb..b5c953a4d97e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q56.plan.txt @@ -6,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - partial aggregation over (i_item_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_83) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_83) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_184) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_184) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q57.plan.txt index 26f16c8477dc..25bb57998645 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q57.plan.txt @@ -4,58 +4,64 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q58.plan.txt index 192c6d359edd..7bd45a0070f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q58.plan.txt @@ -7,28 +7,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (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, []) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_69) local exchange (GATHER, SINGLE, []) @@ -36,53 +39,59 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_69) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_68"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_190"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q59.plan.txt index 7f584558007b..133dd26321d0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q59.plan.txt @@ -4,46 +4,53 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_52"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_52) + 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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q60.plan.txt index 4616b0ad95b6..70f98c7eb54e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q60.plan.txt @@ -6,74 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - partial aggregation over (i_item_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_83) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_83) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_184) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_184) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q61.plan.txt index ff3d350a643f..9205d0d586d7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q61.plan.txt @@ -6,26 +6,28 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store 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 (REPARTITION, ROUND_ROBIN, []) + 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 @@ -35,23 +37,25 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_14"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q62.plan.txt index 90f04a71a65b..c66895aac6b8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q62.plan.txt @@ -3,21 +3,22 @@ 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"]) - partial aggregation over (sm_type, substr$gid, web_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sm_type, substr$gid, web_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan ship_mode diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q63.plan.txt index 23e28eddc4b7..1732f85548b8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q63.plan.txt @@ -2,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q64.plan.txt index 5b7e66dd9c00..a133a52c71bb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q64.plan.txt @@ -3,166 +3,94 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - 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): + 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, []) 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, 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, ["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, ["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): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) join (INNER, REPLICATED): - scan customer + 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 + 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, []) - 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"]) + 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, []) join (INNER, REPLICATED): - scan store_sales + 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, ["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 (REPARTITION, HASH, ["cd_demo_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + 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): @@ -170,10 +98,100 @@ remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + 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): join (INNER, REPLICATED): - scan household_demographics + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band + 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/iceberg/partitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q65.plan.txt index 0c80486a2845..16047fe4c95e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q65.plan.txt @@ -5,30 +5,34 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q66.plan.txt index b87ae78025f5..aa228d59f437 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q66.plan.txt @@ -6,51 +6,55 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q68.plan.txt index 5feab810ee06..52aa91be911d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q68.plan.txt @@ -4,30 +4,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q69.plan.txt index 4b87cab3294e..8de8d86fbc94 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q69.plan.txt @@ -7,42 +7,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED): 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 (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, ["ss_customer_sk"]) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales + 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 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 + 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + scan catalog_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/partitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q70.plan.txt index 0ba93430749a..542ab85591d7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q70.plan.txt @@ -8,26 +8,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (s_state_53) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (s_state_53) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q72.plan.txt index 9c47f45ae5a1..bdbdae78e9d9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q72.plan.txt @@ -9,34 +9,37 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan customer_demographics 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 - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -45,4 +48,5 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q73.plan.txt index 30d700ed0193..9fa35d4ee8c7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q73.plan.txt @@ -3,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q74.plan.txt index 1faae3d6738a..d9e31e2a2fb7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q74.plan.txt @@ -3,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - 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 (REPARTITION, ROUND_ROBIN, []) + 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_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) + 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, ["ws_bill_customer_sk_351"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - scan customer + 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 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_530"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_507"]) - scan customer + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q75.plan.txt index 72ac1ec6cf65..cb8fd860379c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q75.plan.txt @@ -5,101 +5,119 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - 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 + 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 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q76.plan.txt index 96f6228260dc..c1d7f408a238 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q76.plan.txt @@ -3,38 +3,44 @@ 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"]) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) join (INNER, REPLICATED): - scan store_sales + 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 item + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_77"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q77.plan.txt index f5f8b7b62eea..a5c406789d66 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q77.plan.txt @@ -9,74 +9,80 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - partial aggregation over (ss_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - partial aggregation over (sr_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sr_store_sk) join (INNER, REPLICATED): - scan store_returns + 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 + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - 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 (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 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"]) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - partial aggregation over (ws_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - partial aggregation over (wr_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): - scan web_returns + 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, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q78.plan.txt index 50161185b413..4f6eec106eaa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q78.plan.txt @@ -2,48 +2,56 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 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, []) - 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 + 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 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 date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - 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 (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 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 date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q79.plan.txt index 5efde1d3d8c4..7ae16b239b0c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q79.plan.txt @@ -2,23 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q80.plan.txt index a4b2da9fdab0..cbfdaea7dfa3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q80.plan.txt @@ -15,10 +15,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -41,10 +43,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -67,10 +71,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q81.plan.txt index 5011246b8efd..45022024d774 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q81.plan.txt @@ -4,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q82.plan.txt index 532952036fc3..4232c710d2d6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q82.plan.txt @@ -3,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q83.plan.txt index d5992a1244c4..6023a24c3a48 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q83.plan.txt @@ -7,88 +7,100 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - 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"]) - scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_74) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_74"]) - 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 + 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_131) + final aggregation over (d_date_6) 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"]) + 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 + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) scan item - final aggregation over (i_item_id_201) + join (INNER, PARTITIONED): + final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_201"]) - partial aggregation over (i_item_id_201) + remote exchange (REPARTITION, HASH, ["i_item_id_74"]) + partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_returns + 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_258) + final aggregation over (d_date_131) 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 + 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 + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q85.plan.txt index 2a72b1dbbed5..b5198e5c38f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q85.plan.txt @@ -3,36 +3,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - partial aggregation over (r_reason_desc) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (r_reason_desc) join (INNER, REPLICATED): - scan customer_address + 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 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"]) - 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 + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q86.plan.txt index cca927926827..a3b94e200aa6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q86.plan.txt @@ -5,13 +5,14 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q87.plan.txt index 9f02b3c87024..8d422b83cadf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q87.plan.txt @@ -11,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q89.plan.txt index 396d705bbc38..b5495ae3801b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q89.plan.txt @@ -2,20 +2,22 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q91.plan.txt index 750e4be2dcb9..2da8cda3eecd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q91.plan.txt @@ -8,28 +8,32 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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, ["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"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q92.plan.txt index 2dfed78f9a62..d4858936dcb2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q92.plan.txt @@ -7,23 +7,25 @@ final aggregation over () final aggregation over (ws_item_sk_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_6"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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/partitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q93.plan.txt index ca9dc7543db7..a97d8ef18468 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q93.plan.txt @@ -7,10 +7,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q94.plan.txt index 2341a65cbe9b..24b3ed828774 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q94.plan.txt @@ -7,28 +7,31 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - partial aggregation over (wr_order_number) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_22"]) - 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): + 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, []) join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q95.plan.txt index 5e42003378f2..b5f1e9ba58e9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q95.plan.txt @@ -9,35 +9,41 @@ 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_136"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_58"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q97.plan.txt index 7b9c83293534..24bc0bdee176 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q97.plan.txt @@ -6,18 +6,20 @@ 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"]) - 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 + 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 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"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q98.plan.txt index 9775b6c511f5..e5840c437809 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q98.plan.txt @@ -3,16 +3,18 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q99.plan.txt index 7f8838ac985a..652884fb1181 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/partitioned/q99.plan.txt @@ -3,21 +3,22 @@ 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"]) - partial aggregation over (cc_name, sm_type, substr$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cc_name, sm_type, substr$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q01.plan.txt index 114fb89fe851..56f3c72ccdcc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q01.plan.txt @@ -4,37 +4,42 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_customer_sk"]) - join (INNER, REPLICATED): - final aggregation over (sr_customer_sk, sr_store_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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q02.plan.txt index f26bc3efa106..a9e4ac5bcfad 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q02.plan.txt @@ -6,39 +6,44 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_week_seq_20"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_219"]) - join (INNER, PARTITIONED): - final aggregation over (d_week_seq_124) + 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 local exchange (GATHER, SINGLE, []) - 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 + remote exchange (REPARTITION, HASH, ["d_week_seq_169"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q03.plan.txt index 70aae3fa1bdd..4728fd4dc83f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q03.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q04.plan.txt index 56356d683fe6..61aa19348540 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q04.plan.txt @@ -4,99 +4,117 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_848"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_1568"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_1544"]) - scan customer + 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 local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_548"]) - 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_1268"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_1244"]) - scan customer + 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"]) - 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"]) - 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 (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) 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_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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPARTITION, HASH, ["ss_customer_sk_184"]) + 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"]) - scan customer + remote exchange (REPARTITION, HASH, ["c_customer_sk_161"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q05.plan.txt index a837004a655b..ded90e2ed237 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q05.plan.txt @@ -45,10 +45,12 @@ local exchange (GATHER, SINGLE, []) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk_82", "ws_order_number_96"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q06.plan.txt index fb3592d65b6e..1c7ee06ff527 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q06.plan.txt @@ -11,27 +11,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -40,8 +44,9 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category_50) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_category_50"]) - partial aggregation over (i_category_50) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q07.plan.txt index 32f4763b443d..d209bd09ae61 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q07.plan.txt @@ -3,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): 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 date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q08.plan.txt index 45b29afc325b..998b0b959fcb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q08.plan.txt @@ -6,31 +6,36 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_store_name) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["substr_34"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (ca_zip_26) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ca_zip_26) + scan customer_address remote exchange (REPARTITION, HASH, ["ca_zip_29"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q10.plan.txt index 7d60d2a5fdd5..a53dd7834678 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q10.plan.txt @@ -3,48 +3,54 @@ 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"]) - 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - 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, ["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, []) + 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"]) - 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 + 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) local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q11.plan.txt index b0c5b40468d0..9e37382a600e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q11.plan.txt @@ -3,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_79"]) - 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 (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) 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) + 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, ["ws_bill_customer_sk_381"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_357"]) - scan customer + 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 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, ["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 (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_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) + 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, ["ws_bill_customer_sk_574"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_550"]) - scan customer + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q12.plan.txt index eb114fe9092d..609e3537018c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q12.plan.txt @@ -2,17 +2,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q13.plan.txt index a4d7d7d75438..30470310dca0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q13.plan.txt @@ -8,20 +8,23 @@ final aggregation over () join (INNER, REPLICATED): 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 (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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q14.plan.txt index c8326b25e125..21eb6df883d1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q14.plan.txt @@ -9,60 +9,66 @@ 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"]) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) join (INNER, REPLICATED): - scan store_sales + 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -90,60 +96,66 @@ 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"]) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) 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, []) - 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -171,60 +183,66 @@ 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"]) - partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand_id_1052, i_category_id_1056, i_class_id_1054) join (INNER, REPLICATED): - scan web_sales + 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 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 + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q15.plan.txt index 56302f05c739..b22f9d12537c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q15.plan.txt @@ -6,16 +6,19 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q16.plan.txt index 5b6df301073a..d45ed9289047 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q16.plan.txt @@ -7,28 +7,31 @@ final aggregation over () 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 + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cr_order_number) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_order_number_23"]) - 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): + 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, []) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + scan call_center diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q17.plan.txt index 49c98c0b8dda..7a9af86c0c6c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q17.plan.txt @@ -6,32 +6,36 @@ 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q18.plan.txt index 5f2dfaaeb4fa..7f9cbb3ebf63 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q18.plan.txt @@ -4,33 +4,35 @@ 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) - 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): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["cs_bill_customer_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 customer_demographics + 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"]) + 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"]) join (INNER, PARTITIONED): - 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 + remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q19.plan.txt index c500acf6fca3..967aa874de69 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q19.plan.txt @@ -7,23 +7,26 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 item + 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/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q20.plan.txt index 852a9d2a3e5c..2dce6eadbb48 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q20.plan.txt @@ -2,16 +2,18 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q21.plan.txt index 50e40917c430..e8067ff5be06 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q21.plan.txt @@ -3,17 +3,18 @@ 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"]) - partial aggregation over (i_item_id, w_warehouse_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id, w_warehouse_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan inventory + join (INNER, REPLICATED): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q23.plan.txt index c206b58e5ccb..1ef6f718b353 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q23.plan.txt @@ -4,31 +4,35 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk"]) - partial aggregation over (ss_item_sk) - final aggregation over (d_date_6, ss_item_sk, substr$gid) + remote exchange (REPARTITION, HASH, ["cs_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan catalog_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -36,10 +40,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_43) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_43"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -51,42 +57,48 @@ final aggregation over () partial aggregation over (ss_customer_sk_71) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_71"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_93"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, PARTITIONED): - final aggregation over (ss_item_sk_185) + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPARTITION, HASH, ["ws_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + join (INNER, REPLICATED): + scan web_sales local exchange (GATHER, SINGLE, []) - 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 + 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) @@ -94,10 +106,12 @@ final aggregation over () partial aggregation over (ss_customer_sk_274) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_274"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_296"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -109,11 +123,13 @@ final aggregation over () partial aggregation over (ss_customer_sk_322) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_322"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_344"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q24.plan.txt index 14c2547a1059..f9922ec7376e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q24.plan.txt @@ -5,35 +5,40 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -52,19 +57,23 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk_13", "ss_ticket_number_20"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk_38", "sr_ticket_number_45"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q25.plan.txt index 8eedb8e05888..4dcfe955597d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q25.plan.txt @@ -7,31 +7,35 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + 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/unpartitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q26.plan.txt index 555bb67d3baa..dc978ab9fbae 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q26.plan.txt @@ -3,21 +3,22 @@ local exchange (GATHER, SINGLE, []) 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): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q27.plan.txt index b637b1d27f97..22ec76d44200 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/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) - 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, 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 - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q28.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q28.plan.txt index 8f8331d2d85d..fcb9b4b7dd71 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q28.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q28.plan.txt @@ -9,39 +9,45 @@ cross join: partial aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_list_price"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q29.plan.txt index 8eedb8e05888..4dcfe955597d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q29.plan.txt @@ -7,31 +7,35 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan catalog_sales + 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 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 (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 + 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/unpartitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q30.plan.txt index ba844c8c4e25..740f81a2504b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q30.plan.txt @@ -4,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["wr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q31.plan.txt index 2eea355e464a..16b1e2940a8f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q31.plan.txt @@ -10,30 +10,35 @@ 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"]) - join (INNER, REPLICATED): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_59"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_140", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - 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 + 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 join (INNER, PARTITIONED): final aggregation over (ca_county_276, d_qoy_249, d_year_245) local exchange (GATHER, SINGLE, []) @@ -41,58 +46,68 @@ 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"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_269"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_county_361", NullableValue{type=integer, value=2000}, NullableValue{type=integer, value=2}]) - final aggregation over (ca_county_361, d_qoy_334, d_year_330) + 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_361", "d_qoy_334", "d_year_330"]) - partial aggregation over (ca_county_361, d_qoy_334, d_year_330) + 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, ["ws_bill_addr_sk_295"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_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 + 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) + 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"]) + 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_184"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q32.plan.txt index d2a7e4049c81..49bb921af9d5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q32.plan.txt @@ -7,23 +7,25 @@ final aggregation over () final aggregation over (cs_item_sk_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_item_sk_19"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan catalog_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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/unpartitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q33.plan.txt index 1b2427d4e654..2502d90464f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q33.plan.txt @@ -6,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_manufact_id_6"]) - partial aggregation over (i_manufact_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_95) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_95) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_manufact_id_196) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_manufact_id_196) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q34.plan.txt index 8b6f823e1fe3..861d0f0bd0d8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q34.plan.txt @@ -3,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + scan household_demographics diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q35.plan.txt index f94ec8be8411..8353da6955f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q35.plan.txt @@ -11,39 +11,45 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_cdemo_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer 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 (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, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + scan catalog_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/unpartitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q36.plan.txt index d640f5ac0d14..c7aa800fe3c9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q36.plan.txt @@ -5,17 +5,18 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 store + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q37.plan.txt index 2582d9fa4122..1cad16277680 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q37.plan.txt @@ -3,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q38.plan.txt index 9f02b3c87024..8d422b83cadf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q38.plan.txt @@ -11,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q39.plan.txt index 657fe01edba1..0d241239eb7b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q39.plan.txt @@ -3,39 +3,43 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk", "inv_warehouse_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["inv_item_sk_9", "inv_warehouse_sk_10"]) - 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): + 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) join (INNER, REPLICATED): - scan inventory + 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 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, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q40.plan.txt index 266142fc4c1a..62288a2c3dc5 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q40.plan.txt @@ -9,10 +9,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q41.plan.txt index 58a96c8d1ec2..c67211745712 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q41.plan.txt @@ -3,10 +3,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_product_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_product_name"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q42.plan.txt index 17cc12f10e9e..d6a3c8cac82b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q42.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_category, i_category_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_category, i_category_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q43.plan.txt index 9d9e36441a12..53c0550a78af 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q43.plan.txt @@ -3,17 +3,19 @@ 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"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q44.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q44.plan.txt index d84084843142..e553c8e97eed 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q44.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q44.plan.txt @@ -4,42 +4,48 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["rank"]) - 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 (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) cross join: - final aggregation over (ss_item_sk_28) + final aggregation over (ss_item_sk) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_item_sk_28"]) - partial aggregation over (ss_item_sk_28) - scan store_sales + remote exchange (REPARTITION, HASH, ["ss_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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_60) + final aggregation over (ss_store_sk_7) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ss_store_sk_60"]) - partial aggregation over (ss_store_sk_60) - scan store_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q45.plan.txt index 5b30cdf7426c..b275b76606e4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q45.plan.txt @@ -10,20 +10,23 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_bill_customer_sk"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q46.plan.txt index e8aa3db7886c..6ddeddd43120 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q46.plan.txt @@ -4,30 +4,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_11"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["ca_address_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q47.plan.txt index 2d63bde52f55..93faff459e9c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q47.plan.txt @@ -4,58 +4,64 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q48.plan.txt index 7eb552bd09e3..4e0866c981c0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q48.plan.txt @@ -7,17 +7,20 @@ final aggregation over () remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) 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 (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"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cd_demo_sk"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q49.plan.txt index 6f687392953d..66381314bec4 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q49.plan.txt @@ -3,56 +3,65 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["cr_item_sk", "expr_66", "expr_70", "rank_68", "rank_69"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["expr_115", "expr_120", "rank_117", "rank_118", "sr_item_sk"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q50.plan.txt index c2a9d481bf10..c28e1f11bc9a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q50.plan.txt @@ -8,14 +8,16 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk", "ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales 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 (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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q51.plan.txt index ed5e8607b930..06b2887fa30d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q51.plan.txt @@ -2,26 +2,31 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q52.plan.txt index 33752e693e6c..ab4fb497ba27 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q52.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (d_year, i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q53.plan.txt index cd10ab3698e6..2206a00753f9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q53.plan.txt @@ -2,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q54.plan.txt index 23ee4dacf66b..73276be84ef1 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q54.plan.txt @@ -3,64 +3,71 @@ local exchange (GATHER, SINGLE, []) final aggregation over (expr$gid) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr$gid"]) - 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 (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 local exchange (GATHER, SINGLE, []) - 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): + 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) join (INNER, REPLICATED, can skip output duplicates): - remote exchange (REPARTITION, ROUND_ROBIN, []) - scan catalog_sales - scan web_sales + scan customer 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, []) + 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 (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) + remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["expr_104"]) - partial aggregation over (expr_104) - scan date_dim + 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q55.plan.txt index 0322599f01e5..ad84f78b05f8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q55.plan.txt @@ -3,13 +3,14 @@ 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"]) - partial aggregation over (i_brand, i_brand_id) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): - scan store_sales + 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 item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q56.plan.txt index e96c8c31aabb..b5c953a4d97e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q56.plan.txt @@ -6,77 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - partial aggregation over (i_item_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_83) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_83) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_184) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_184) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q57.plan.txt index 26f16c8477dc..25bb57998645 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q57.plan.txt @@ -4,58 +4,64 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_102", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cc_name_232", "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): + 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) join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q58.plan.txt index 192c6d359edd..7bd45a0070f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q58.plan.txt @@ -7,28 +7,31 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (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, []) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item join (INNER, PARTITIONED): final aggregation over (i_item_id_69) local exchange (GATHER, SINGLE, []) @@ -36,53 +39,59 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id_69) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_68"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim + 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_190"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q59.plan.txt index ae30fe8f578e..72c157ff0704 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q59.plan.txt @@ -4,46 +4,53 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["expr_203", "s_store_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_store_sk_53"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - final aggregation over (d_week_seq_75, ss_store_sk_53) + 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 local exchange (GATHER, SINGLE, []) - 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 + 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"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["s_store_id_118"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q60.plan.txt index 4616b0ad95b6..70f98c7eb54e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q60.plan.txt @@ -6,74 +6,83 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_id_6"]) - partial aggregation over (i_item_id_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_83) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_83) join (INNER, REPLICATED): 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 + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 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"]) - partial aggregation over (i_item_id_184) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (i_item_id_184) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q61.plan.txt index f5183d67d01e..2aa259b1de4f 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q61.plan.txt @@ -6,26 +6,28 @@ cross join: join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store 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 (REPARTITION, ROUND_ROBIN, []) + 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 @@ -35,23 +37,25 @@ cross join: partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk_15"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan store_sales + 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 + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_98"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q62.plan.txt index 90f04a71a65b..c66895aac6b8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q62.plan.txt @@ -3,21 +3,22 @@ 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"]) - partial aggregation over (sm_type, substr$gid, web_name) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sm_type, substr$gid, web_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 date_dim + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan web_site local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan web_site - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan ship_mode diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q63.plan.txt index 23e28eddc4b7..1732f85548b8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q63.plan.txt @@ -2,20 +2,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q64.plan.txt index b3b3aa65cb46..1d630ce9b299 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q64.plan.txt @@ -3,166 +3,94 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_name", "s_zip", "ss_item_sk"]) - 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): + 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, []) 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, 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, ["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, ["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): + remote exchange (REPARTITION, HASH, ["ss_cdemo_sk"]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) join (INNER, REPLICATED): - scan customer + 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 + 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, []) - 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"]) + 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, []) join (INNER, REPLICATED): - scan store_sales + 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, ["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 (REPARTITION, HASH, ["cd_demo_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["cd_demo_sk_389"]) - scan customer_demographics + remote exchange (REPLICATE, BROADCAST, []) + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion + 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): @@ -170,10 +98,100 @@ remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) + 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): join (INNER, REPLICATED): - scan household_demographics + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan income_band + 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/iceberg/unpartitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q65.plan.txt index d464032d91fc..5d8a33cc5245 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q65.plan.txt @@ -5,30 +5,34 @@ 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_store_sk"]) - scan store + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q66.plan.txt index b87ae78025f5..aa228d59f437 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q66.plan.txt @@ -6,51 +6,55 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + 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"]) - 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): + 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) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan warehouse diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q68.plan.txt index 5feab810ee06..52aa91be911d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q68.plan.txt @@ -4,30 +4,35 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_current_addr_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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, []) join (INNER, REPLICATED): - scan store_sales + 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 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 household_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk_12"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_address diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q69.plan.txt index 4b87cab3294e..8de8d86fbc94 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q69.plan.txt @@ -7,42 +7,47 @@ local exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED): 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 (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, ["ss_customer_sk"]) - partial aggregation over (ss_customer_sk) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales + 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 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 + 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) + 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) join (INNER, REPLICATED, can skip output duplicates): - scan web_sales + scan catalog_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/unpartitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q70.plan.txt index 0ba93430749a..542ab85591d7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q70.plan.txt @@ -8,26 +8,28 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, s_county$gid, s_state$gid_92) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["s_state"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan store_sales + 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 store + 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"]) - partial aggregation over (s_state_53) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (s_state_53) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q72.plan.txt index 9c47f45ae5a1..bdbdae78e9d9 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q72.plan.txt @@ -9,34 +9,37 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["inv_item_sk"]) - join (INNER, REPLICATED): - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - scan catalog_sales + join (INNER, REPLICATED): + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan customer_demographics 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 - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_item_sk"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -45,4 +48,5 @@ local exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q73.plan.txt index 30d700ed0193..9fa35d4ee8c7 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q73.plan.txt @@ -3,23 +3,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q74.plan.txt index 3000b513f3ec..23b39e29816e 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q74.plan.txt @@ -3,66 +3,78 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["c_customer_id_69"]) - 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 (REPARTITION, ROUND_ROBIN, []) + 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_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) + 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, ["ws_bill_customer_sk_352"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["c_customer_sk_328"]) - scan customer + 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 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"]) - 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 (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_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) + 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, ["ws_bill_customer_sk_531"]) - join (INNER, REPLICATED): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_507"]) - scan customer + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q75.plan.txt index fef6864e7254..7e7a9117d8eb 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q75.plan.txt @@ -5,101 +5,119 @@ 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_28", "i_category_id_32", "i_class_id_30", "i_manufact_id_34"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_88", "i_category_id_92", "i_class_id_90", "i_manufact_id_94"]) - 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 + 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 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"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_308", "i_category_id_312", "i_class_id_310", "i_manufact_id_314"]) - 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 + 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 remote exchange (REPARTITION, HASH, ["i_brand_id_422", "i_category_id_426", "i_class_id_424", "i_manufact_id_428"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q76.plan.txt index 96f6228260dc..c1d7f408a238 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q76.plan.txt @@ -3,38 +3,44 @@ 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"]) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) join (INNER, REPLICATED): - scan store_sales + 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 item + 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"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_sold_date_sk"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["d_date_sk_41"]) - scan date_dim + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - join (INNER, REPLICATED): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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_77"]) - scan item + local exchange (REPARTITION, ROUND_ROBIN, []) + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q77.plan.txt index f5f8b7b62eea..a5c406789d66 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q77.plan.txt @@ -9,74 +9,80 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ss_store_sk"]) - partial aggregation over (ss_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): - scan store_sales + 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 store + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_store_sk"]) - partial aggregation over (sr_store_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (sr_store_sk) join (INNER, REPLICATED): - scan store_returns + 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 + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_call_center_sk"]) - 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 (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 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"]) - partial aggregation over (cr_call_center_sk) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 join (LEFT, PARTITIONED): final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_web_page_sk"]) - partial aggregation over (ws_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_web_page_sk"]) - partial aggregation over (wr_web_page_sk) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): - scan web_returns + 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, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + scan web_page diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q78.plan.txt index 50161185b413..4f6eec106eaa 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q78.plan.txt @@ -2,48 +2,56 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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 (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 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, []) - 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 + 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 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 date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cs_bill_customer_sk"]) - 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 (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 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 date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q79.plan.txt index 5efde1d3d8c4..7ae16b239b0c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q79.plan.txt @@ -2,23 +2,26 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_customer_sk"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 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 (REPLICATE, BROADCAST, []) - scan store + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q80.plan.txt index a4b2da9fdab0..cbfdaea7dfa3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q80.plan.txt @@ -15,10 +15,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -41,10 +43,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["cs_item_sk", "cs_order_number"]) - scan catalog_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["cr_item_sk", "cr_order_number"]) - scan catalog_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -67,10 +71,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["ws_item_sk", "ws_order_number"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_item_sk", "wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q81.plan.txt index 03e019fc0d5f..f0c2e9ce4e42 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q81.plan.txt @@ -4,47 +4,54 @@ local exchange (GATHER, SINGLE, []) join (LEFT, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - 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 (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 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 (REPARTITION, ROUND_ROBIN, []) + 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"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q82.plan.txt index 532952036fc3..4232c710d2d6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q82.plan.txt @@ -3,17 +3,18 @@ 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"]) - 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): + 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, []) join (INNER, REPLICATED, can skip output duplicates): - scan inventory + join (INNER, REPLICATED, can skip output duplicates): + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q83.plan.txt index d5992a1244c4..6023a24c3a48 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q83.plan.txt @@ -7,88 +7,100 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_item_id) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["sr_item_sk"]) - 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"]) - scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_74) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_74"]) - 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 + 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_131) + final aggregation over (d_date_6) 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"]) + 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 + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) scan item - final aggregation over (i_item_id_201) + join (INNER, PARTITIONED): + final aggregation over (i_item_id_74) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["i_item_id_201"]) - partial aggregation over (i_item_id_201) + remote exchange (REPARTITION, HASH, ["i_item_id_74"]) + partial aggregation over (i_item_id_74) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, ["i_item_sk_200"]) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["wr_item_sk"]) + remote exchange (REPARTITION, HASH, ["cr_item_sk"]) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan web_returns + 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_258) + final aggregation over (d_date_131) 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 + 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 + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, ["i_item_sk_73"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q85.plan.txt index 2a72b1dbbed5..b5198e5c38f0 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q85.plan.txt @@ -3,36 +3,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (r_reason_desc) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["r_reason_desc"]) - partial aggregation over (r_reason_desc) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (r_reason_desc) join (INNER, REPLICATED): - scan customer_address + 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 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"]) - 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 + scan reason diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q86.plan.txt index cca927926827..a3b94e200aa6 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q86.plan.txt @@ -5,13 +5,14 @@ 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"]) - partial aggregation over (groupid, i_category$gid, i_class$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (groupid, i_category$gid, i_class$gid) join (INNER, REPLICATED): - scan web_sales + 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 date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q87.plan.txt index 9f02b3c87024..8d422b83cadf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q87.plan.txt @@ -11,14 +11,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -26,14 +28,16 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_42"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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, []) @@ -41,11 +45,13 @@ 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"]) - join (INNER, REPLICATED, can skip output duplicates): - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["c_customer_sk_94"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q89.plan.txt index 396d705bbc38..b5495ae3801b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q89.plan.txt @@ -2,20 +2,22 @@ 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"]) - 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): + 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) join (INNER, REPLICATED): - scan store_sales + 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 (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + scan store diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q91.plan.txt index 750e4be2dcb9..2da8cda3eecd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q91.plan.txt @@ -8,28 +8,32 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["cr_returning_customer_sk"]) - join (INNER, REPLICATED): - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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, ["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"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan customer_demographics + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ca_address_sk"]) - scan customer_address + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q92.plan.txt index 55eb5a4a778f..03866eaa7ffd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q92.plan.txt @@ -7,23 +7,25 @@ final aggregation over () final aggregation over (ws_item_sk_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_item_sk_7"]) - partial aggregation over (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): 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"]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): - scan web_sales + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + 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/unpartitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q93.plan.txt index ca9dc7543db7..a97d8ef18468 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q93.plan.txt @@ -7,10 +7,12 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["ss_item_sk", "ss_ticket_number"]) - scan store_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["sr_item_sk", "sr_ticket_number"]) - scan store_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q94.plan.txt index 821a5b58e819..cf882450c774 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q94.plan.txt @@ -7,28 +7,31 @@ final aggregation over () final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - partial aggregation over (wr_order_number) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (wr_order_number) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_23"]) - 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): + 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, []) join (INNER, REPLICATED): - scan web_sales + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - 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 + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q95.plan.txt index aaf3edd0ed42..21e3f3d023f2 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q95.plan.txt @@ -9,35 +9,41 @@ 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["wr_order_number"]) - scan web_returns + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_137"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number_59"]) - scan web_sales + local exchange (REPARTITION, ROUND_ROBIN, []) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["ws_order_number"]) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): join (INNER, REPLICATED): - scan web_sales + 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 customer_address + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_site + scan web_site diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q97.plan.txt index 7b9c83293534..24bc0bdee176 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q97.plan.txt @@ -6,18 +6,20 @@ 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"]) - 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 + 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 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"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q98.plan.txt index 9775b6c511f5..e5840c437809 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q98.plan.txt @@ -3,16 +3,18 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, ROUND_ROBIN, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["i_class"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q99.plan.txt index 7f8838ac985a..652884fb1181 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpcds/iceberg/unpartitioned/q99.plan.txt @@ -3,21 +3,22 @@ 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"]) - partial aggregation over (cc_name, sm_type, substr$gid) - join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (cc_name, sm_type, substr$gid) join (INNER, REPLICATED): join (INNER, REPLICATED): 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 + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan call_center + scan call_center 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 d632b41b395a..72eae868fa59 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,5 +4,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - partial aggregation over (linestatus, returnflag) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 b593422eaaf0..73ba5b2465e3 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,40 +6,43 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_23) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_23"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - join (INNER, REPLICATED): - scan partsupp - 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 + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + 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 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 4a9e1323a310..1c9dd5870ba2 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,12 +5,15 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_5, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 5d0194bd4f62..ee8edb198516 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,13 +4,15 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - 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 + 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 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 cb04cfef04bd..de2c161dfdfd 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,24 +7,27 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (name_22) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["nationkey_14", "orderkey_5"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e334b46ac4d6..27c32914ba01 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,22 +9,25 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - 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, []) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan supplier + scan customer 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 34062f9cadd6..4241a644111f 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,31 +9,35 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - join (INNER, REPLICATED): - scan lineitem - 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 + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan nation + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_11"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_17"]) - 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 (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 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 8b8f3844d639..b05841b29304 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,19 +13,24 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_12"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_17"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 4b45077ce17c..f27e3195bc8a 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,16 +6,19 @@ 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"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_6"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 bfc5af4b5bb2..70b8b0f9b94c 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,16 +5,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - 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 (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 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 7fb5b836dfa9..36ac0385529a 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,7 +7,9 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e30665fcb33b..c277ff8900bc 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,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - 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 + 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 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 e46c0418a759..d2330a52a902 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,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 c95f702a6873..99d24e6b9a47 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,8 +9,9 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - partial aggregation over (suppkey_0) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -20,5 +21,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_7"]) - partial aggregation over (suppkey_7) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 97eea04d0aa1..9c0d1a7a90d3 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,18 +4,22 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - 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 + 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 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 0d192b20aa3d..3941a8a3ddc9 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,8 +14,9 @@ final aggregation over () final aggregation over (partkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_6"]) - partial aggregation over (partkey_6) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 463aec3e5325..bb9785189f52 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,21 +5,25 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_5, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 fa00639798fe..4ba17d407e65 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,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 89b191d4eafd..ada99e1e7958 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,32 +5,37 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) - partial aggregation over (partkey_21, suppkey_22) - scan lineitem + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 1460eaa40b15..8dff250145df 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,30 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - 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): + 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) 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 + 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 local exchange (GATHER, SINGLE, []) - 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 + 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 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 0cc72848180d..dd43ccd8d138 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,17 +7,19 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - 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 + 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 final aggregation over (custkey_11) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - partial aggregation over (custkey_11) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 d632b41b395a..72eae868fa59 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,5 +4,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - partial aggregation over (linestatus, returnflag) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 b593422eaaf0..73ba5b2465e3 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,40 +6,43 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_23) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_23"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey_6"]) - join (INNER, REPLICATED): - scan partsupp - 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 + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - scan nation - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan region + 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 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 4a9e1323a310..1c9dd5870ba2 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,12 +5,15 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_5, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 91dcfd113534..9964bd40d4f8 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,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - partial aggregation over (orderkey_0) - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 11e54014a4ca..784940efa7ec 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,24 +7,27 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (name_21) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["nationkey_13", "orderkey_5"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 4a90291c58c0..19e22ce67b58 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,24 +7,27 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (expr$gid, name_21, name_27) 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 (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"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 c196a44d68e0..01b05dbfabd0 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,29 +11,33 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_11"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - 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, []) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan nation + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan region + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 24be20eb1056..54087e8f540c 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,19 +13,24 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_12"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_17"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 60df9b930b5b..2b2a513ac7ca 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,15 +6,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_5"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 bfc5af4b5bb2..70b8b0f9b94c 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,16 +5,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - 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 (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 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 7fb5b836dfa9..36ac0385529a 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,7 +7,9 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 e30665fcb33b..c277ff8900bc 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,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - 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 + 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 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 e46c0418a759..d2330a52a902 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,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 02ee343d1df5..da1195652f8d 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,12 +4,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - partial aggregation over (suppkey_0) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -19,5 +21,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_7"]) - partial aggregation over (suppkey_7) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 97eea04d0aa1..9c0d1a7a90d3 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,18 +4,22 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - 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 + 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 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 72a457f937bf..69b972107f29 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,15 +7,17 @@ final aggregation over () final aggregation over (partkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_6"]) - partial aggregation over (partkey_6) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (partkey_6) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + 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 (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 b9b41cc41b90..2124161181e2 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,20 +5,24 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (custkey_0, name, orderdate, orderkey_5, totalprice) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_5"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + 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"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders single aggregation over (orderkey_10) final aggregation over (orderkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_10"]) - partial aggregation over (orderkey_10) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (orderkey_10) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 fa00639798fe..4ba17d407e65 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,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 89b191d4eafd..ada99e1e7958 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,32 +5,37 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_21", "suppkey_22"]) - partial aggregation over (partkey_21, suppkey_22) - scan lineitem + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 d62c188bfd61..e63e68a5a27f 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,31 +3,36 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - 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 + 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 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 0cc72848180d..dd43ccd8d138 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,17 +7,19 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - 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 + 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 final aggregation over (custkey_11) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_11"]) - partial aggregation over (custkey_11) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (custkey_11) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q01.plan.txt index d632b41b395a..72eae868fa59 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q01.plan.txt @@ -4,5 +4,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - partial aggregation over (linestatus, returnflag) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q02.plan.txt index 358da707a03d..dc93bb7bc1fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q02.plan.txt @@ -6,41 +6,45 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - 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 (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 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"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q03.plan.txt index f5cf1c82bd37..4e117498e676 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q03.plan.txt @@ -5,12 +5,15 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q04.plan.txt index 91dcfd113534..9964bd40d4f8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q04.plan.txt @@ -4,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - partial aggregation over (orderkey_0) - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q05.plan.txt index 8d476b717a72..5ad7aaebee00 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q05.plan.txt @@ -9,23 +9,27 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q07.plan.txt index 64d5f1fecd08..727ba23e2f4b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q07.plan.txt @@ -9,22 +9,25 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 (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, ["custkey_8"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q08.plan.txt index 88f8a11f4445..724fc473a19a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q08.plan.txt @@ -12,28 +12,32 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - 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, []) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan nation + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan region + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q09.plan.txt index a55a51f232bd..935c5e847127 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q09.plan.txt @@ -13,19 +13,24 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q10.plan.txt index 7ab0ad56a3f1..c084f568796c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q10.plan.txt @@ -6,15 +6,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q11.plan.txt index bfc5af4b5bb2..70b8b0f9b94c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q11.plan.txt @@ -5,16 +5,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q12.plan.txt index 7fb5b836dfa9..36ac0385529a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q12.plan.txt @@ -7,7 +7,9 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q13.plan.txt index e30665fcb33b..c277ff8900bc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q13.plan.txt @@ -4,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q14.plan.txt index e46c0418a759..d2330a52a902 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q14.plan.txt @@ -4,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q15.plan.txt index b9a65952a467..8d72d43209bf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q15.plan.txt @@ -4,12 +4,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - partial aggregation over (suppkey_0) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -19,5 +21,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - partial aggregation over (suppkey_6) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q16.plan.txt index bd14aff914fd..c94f3d6e659d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q16.plan.txt @@ -4,18 +4,22 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q17.plan.txt index 30539653b56b..1c0100d18830 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q17.plan.txt @@ -7,15 +7,17 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - partial aggregation over (partkey_5) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + 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 (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q18.plan.txt index 296a79e4c9cf..741668695fd3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q18.plan.txt @@ -9,16 +9,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - partial aggregation over (orderkey_8) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q19.plan.txt index e46c0418a759..d2330a52a902 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q19.plan.txt @@ -4,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q20.plan.txt index cb79004dac3a..78592280adc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q20.plan.txt @@ -5,32 +5,37 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q21.plan.txt index 313eb576ce37..fdc46b4f0e52 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q21.plan.txt @@ -3,30 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - 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 (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 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_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 + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q22.plan.txt index eba30d3c79d1..12c1be1b0be3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/partitioned/q22.plan.txt @@ -7,17 +7,19 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - 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 + 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 final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - partial aggregation over (custkey_10) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (custkey_10) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q01.plan.txt index d632b41b395a..72eae868fa59 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q01.plan.txt @@ -4,5 +4,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (linestatus, returnflag) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["linestatus", "returnflag"]) - partial aggregation over (linestatus, returnflag) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (linestatus, returnflag) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q02.plan.txt index 358da707a03d..dc93bb7bc1fd 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q02.plan.txt @@ -6,41 +6,45 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey_19) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_19"]) - 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 (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 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"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q03.plan.txt index f5cf1c82bd37..4e117498e676 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q03.plan.txt @@ -5,12 +5,15 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (orderdate, orderkey_4, shippriority) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q04.plan.txt index 91dcfd113534..9964bd40d4f8 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q04.plan.txt @@ -4,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (orderpriority) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderpriority"]) - partial aggregation over (orderpriority) - join (INNER, PARTITIONED): - final aggregation over (orderkey_0) + 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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey_0"]) - partial aggregation over (orderkey_0) - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + remote exchange (REPARTITION, HASH, ["orderkey"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q05.plan.txt index 8d476b717a72..5ad7aaebee00 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q05.plan.txt @@ -9,23 +9,27 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey", "suppkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["nationkey_11", "suppkey_8"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q07.plan.txt index 64d5f1fecd08..727ba23e2f4b 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q07.plan.txt @@ -9,22 +9,25 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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 (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, ["custkey_8"]) - join (INNER, REPLICATED): - scan customer - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q08.plan.txt index 88f8a11f4445..724fc473a19a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q08.plan.txt @@ -12,28 +12,32 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey_9"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey"]) - 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, []) + local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - scan nation + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan region + 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 local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q09.plan.txt index a55a51f232bd..935c5e847127 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q09.plan.txt @@ -13,19 +13,24 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_10"]) - scan partsupp + local exchange (REPARTITION, ROUND_ROBIN, []) + scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_14"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q10.plan.txt index 7ab0ad56a3f1..c084f568796c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q10.plan.txt @@ -6,15 +6,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey_4"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q11.plan.txt index bfc5af4b5bb2..70b8b0f9b94c 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q11.plan.txt @@ -5,16 +5,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (partkey) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - 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 (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 local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q12.plan.txt index 7fb5b836dfa9..36ac0385529a 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q12.plan.txt @@ -7,7 +7,9 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_0"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q13.plan.txt index e30665fcb33b..c277ff8900bc 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q13.plan.txt @@ -4,13 +4,16 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (count) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["count"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q14.plan.txt index e46c0418a759..d2330a52a902 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q14.plan.txt @@ -4,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q15.plan.txt index b9a65952a467..8d72d43209bf 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q15.plan.txt @@ -4,12 +4,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["suppkey"]) - scan supplier + local exchange (REPARTITION, ROUND_ROBIN, []) + scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_0"]) - partial aggregation over (suppkey_0) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_0) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -19,5 +21,6 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_6"]) - partial aggregation over (suppkey_6) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (suppkey_6) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q16.plan.txt index bd14aff914fd..c94f3d6e659d 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q16.plan.txt @@ -4,18 +4,22 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (brand, size, type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["brand", "size", "type"]) - 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 + 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 diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q17.plan.txt index 30539653b56b..1c0100d18830 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q17.plan.txt @@ -7,15 +7,17 @@ final aggregation over () final aggregation over (partkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey_5"]) - partial aggregation over (partkey_5) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (partkey_5) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - join (INNER, REPLICATED): - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan part + 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 (REPLICATE, BROADCAST, []) single aggregation over () diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q18.plan.txt index 296a79e4c9cf..741668695fd3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q18.plan.txt @@ -9,16 +9,20 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_0"]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["orderkey"]) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + scan orders single aggregation over (orderkey_8) final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_8"]) - partial aggregation over (orderkey_8) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (orderkey_8) + scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey"]) - scan customer + local exchange (REPARTITION, ROUND_ROBIN, []) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["orderkey_4"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q19.plan.txt index e46c0418a759..d2330a52a902 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q19.plan.txt @@ -4,7 +4,9 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, ["partkey_0"]) - scan part + local exchange (REPARTITION, ROUND_ROBIN, []) + scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["partkey"]) - scan lineitem + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q20.plan.txt index cb79004dac3a..78592280adc3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q20.plan.txt @@ -5,32 +5,37 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (suppkey_5) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey_5"]) - 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 (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 local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, ["partkey_18", "suppkey_19"]) - partial aggregation over (partkey_18, suppkey_19) - scan lineitem + 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 local exchange (GATHER, SINGLE, []) - 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) + remote exchange (REPLICATE, BROADCAST, []) + single aggregation over () + values (1 rows) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["suppkey"]) - join (INNER, REPLICATED): - scan supplier - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan nation + local exchange (REPARTITION, ROUND_ROBIN, []) + 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/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q21.plan.txt index 313eb576ce37..fdc46b4f0e52 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q21.plan.txt @@ -3,30 +3,35 @@ local exchange (GATHER, SINGLE, []) final aggregation over (name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["name"]) - 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 (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 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_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 + remote exchange (REPARTITION, HASH, ["orderkey_32"]) + local exchange (REPARTITION, ROUND_ROBIN, []) + scan lineitem diff --git a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q22.plan.txt index eba30d3c79d1..12c1be1b0be3 100644 --- a/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/presto/tpch/iceberg/unpartitioned/q22.plan.txt @@ -7,17 +7,19 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (substr$gid) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, ["custkey"]) - 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 + 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 final aggregation over (custkey_10) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, ["custkey_10"]) - partial aggregation over (custkey_10) - scan orders + local exchange (REPARTITION, ROUND_ROBIN, []) + partial aggregation over (custkey_10) + scan orders From 9c3a69aa344a38aeb7514dabcb3a9b33261a3b72 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 1 Nov 2022 14:45:27 -0400 Subject: [PATCH 2/5] Drop BroadcastExchanger Broadcast local exchanges are never performed --- .../operator/exchange/BroadcastExchanger.java | 56 ----------- .../operator/exchange/LocalExchange.java | 8 +- .../operator/exchange/TestLocalExchange.java | 99 +------------------ 3 files changed, 6 insertions(+), 157 deletions(-) delete mode 100644 core/trino-main/src/main/java/io/trino/operator/exchange/BroadcastExchanger.java diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/BroadcastExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/BroadcastExchanger.java deleted file mode 100644 index 63d4fdc19a80..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/BroadcastExchanger.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.operator.exchange; - -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ListenableFuture; -import io.trino.operator.exchange.PageReference.PageReleasedListener; -import io.trino.spi.Page; - -import java.util.List; -import java.util.function.Consumer; - -import static java.util.Objects.requireNonNull; - -class BroadcastExchanger - implements LocalExchanger -{ - private final List> buffers; - private final LocalExchangeMemoryManager memoryManager; - private final PageReleasedListener onPageReleased; - - public BroadcastExchanger(List> buffers, LocalExchangeMemoryManager memoryManager) - { - this.buffers = ImmutableList.copyOf(requireNonNull(buffers, "buffers is null")); - this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); - this.onPageReleased = PageReleasedListener.forLocalExchangeMemoryManager(memoryManager); - } - - @Override - public void accept(Page page) - { - PageReference pageReference = new PageReference(page, buffers.size(), onPageReleased); - memoryManager.updateMemoryUsage(pageReference.getRetainedSizeInBytes()); - - for (Consumer buffer : buffers) { - buffer.accept(pageReference); - } - } - - @Override - public ListenableFuture waitForWriting() - { - return memoryManager.getNotFullFuture(); - } -} diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java index f70c6deb5aa8..e9e9236b515f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java @@ -114,13 +114,7 @@ public LocalExchange( .collect(toImmutableList()); this.memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); - if (partitioning.equals(SINGLE_DISTRIBUTION)) { - exchangerSupplier = () -> new BroadcastExchanger(buffers, memoryManager); - } - else if (partitioning.equals(FIXED_BROADCAST_DISTRIBUTION)) { - exchangerSupplier = () -> new BroadcastExchanger(buffers, memoryManager); - } - else if (partitioning.equals(FIXED_ARBITRARY_DISTRIBUTION)) { + if (partitioning.equals(SINGLE_DISTRIBUTION) || partitioning.equals(FIXED_ARBITRARY_DISTRIBUTION)) { exchangerSupplier = () -> new RandomExchanger(buffers, memoryManager); } else if (partitioning.equals(FIXED_PASSTHROUGH_DISTRIBUTION)) { diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index bfe74079103c..5561c01417eb 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -61,7 +61,6 @@ import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; -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; @@ -179,94 +178,6 @@ public void testGatherSingleWriter() }); } - @Test - public void testBroadcast() - { - LocalExchange localExchange = new LocalExchange( - nodePartitioningManager, - SESSION, - 2, - FIXED_BROADCAST_DISTRIBUTION, - ImmutableList.of(), - ImmutableList.of(), - Optional.empty(), - LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, - TYPE_OPERATOR_FACTORY, - WRITER_MIN_SIZE); - - run(localExchange, exchange -> { - assertEquals(exchange.getBufferCount(), 2); - assertExchangeTotalBufferedBytes(exchange, 0); - - LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); - sinkFactory.noMoreSinkFactories(); - LocalExchangeSink sinkA = sinkFactory.createSink(); - assertSinkCanWrite(sinkA); - LocalExchangeSink sinkB = sinkFactory.createSink(); - assertSinkCanWrite(sinkB); - sinkFactory.close(); - - LocalExchangeSource sourceA = getNextSource(exchange); - assertSource(sourceA, 0); - - LocalExchangeSource sourceB = getNextSource(exchange); - assertSource(sourceB, 0); - - sinkA.addPage(createPage(0)); - - assertSource(sourceA, 1); - assertSource(sourceB, 1); - assertExchangeTotalBufferedBytes(exchange, 1); - - sinkA.addPage(createPage(0)); - - assertSource(sourceA, 2); - assertSource(sourceB, 2); - assertExchangeTotalBufferedBytes(exchange, 2); - - assertRemovePage(sourceA, createPage(0)); - assertSource(sourceA, 1); - assertSource(sourceB, 2); - assertExchangeTotalBufferedBytes(exchange, 2); - - assertRemovePage(sourceA, createPage(0)); - assertSource(sourceA, 0); - assertSource(sourceB, 2); - assertExchangeTotalBufferedBytes(exchange, 2); - - sinkA.finish(); - assertSinkFinished(sinkA); - assertExchangeTotalBufferedBytes(exchange, 2); - - sinkB.addPage(createPage(0)); - assertSource(sourceA, 1); - assertSource(sourceB, 3); - assertExchangeTotalBufferedBytes(exchange, 3); - - sinkB.finish(); - assertSinkFinished(sinkB); - assertSource(sourceA, 1); - assertSource(sourceB, 3); - assertExchangeTotalBufferedBytes(exchange, 3); - - assertRemovePage(sourceA, createPage(0)); - assertSourceFinished(sourceA); - assertSource(sourceB, 3); - assertExchangeTotalBufferedBytes(exchange, 3); - - assertRemovePage(sourceB, createPage(0)); - assertRemovePage(sourceB, createPage(0)); - assertSourceFinished(sourceA); - assertSource(sourceB, 1); - assertExchangeTotalBufferedBytes(exchange, 1); - - assertRemovePage(sourceB, createPage(0)); - assertSourceFinished(sourceA); - assertSourceFinished(sourceB); - assertExchangeTotalBufferedBytes(exchange, 0); - }); - } - @Test public void testRandom() { @@ -991,7 +902,7 @@ public void writeUnblockWhenAllReadersFinish() nodePartitioningManager, SESSION, 2, - FIXED_BROADCAST_DISTRIBUTION, + FIXED_ARBITRARY_DISTRIBUTION, ImmutableList.of(), ImmutableList.of(), Optional.empty(), @@ -1038,11 +949,11 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() nodePartitioningManager, SESSION, 2, - FIXED_BROADCAST_DISTRIBUTION, + FIXED_PASSTHROUGH_DISTRIBUTION, ImmutableList.of(), ImmutableList.of(), Optional.empty(), - DataSize.ofBytes(1), + DataSize.ofBytes(2), TYPE_OPERATOR_FACTORY, WRITER_MIN_SIZE); @@ -1072,11 +983,12 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() sinkA.addPage(createPage(0)); ListenableFuture sinkAFuture = assertSinkWriteBlocked(sinkA); + sinkB.addPage(createPage(0)); ListenableFuture sinkBFuture = assertSinkWriteBlocked(sinkB); assertSource(sourceA, 1); assertSource(sourceB, 1); - assertExchangeTotalBufferedBytes(exchange, 1); + assertExchangeTotalBufferedBytes(exchange, 2); sourceA.finish(); assertSource(sourceA, 1); @@ -1085,7 +997,6 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed() assertExchangeTotalBufferedBytes(exchange, 1); assertSource(sourceB, 1); - assertSinkWriteBlocked(sinkA); assertSinkWriteBlocked(sinkB); sourceB.finish(); From 7d7b95e87a7050ccc911fba2c935488b34599dce Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 1 Nov 2022 14:47:02 -0400 Subject: [PATCH 3/5] Optimize LocalExchangeSource Use ArrayDeque to avoid unnecessary allocations associated with a LinkedList based queue --- .../exchange/LocalExchangeSource.java | 31 +++++++++++++------ 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java index 834529e05c67..ba5331839d05 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java @@ -23,10 +23,11 @@ import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.LinkedBlockingDeque; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -41,8 +42,11 @@ public class LocalExchangeSource private final Consumer onFinish; - private final BlockingQueue buffer = new LinkedBlockingDeque<>(); + @GuardedBy("this") + private final Queue buffer = new ArrayDeque<>(); + private final AtomicLong bufferedBytes = new AtomicLong(); + private final AtomicInteger bufferedPages = new AtomicInteger(); @Nullable @GuardedBy("this") @@ -59,7 +63,7 @@ public LocalExchangeBufferInfo getBufferInfo() { // This must be lock free to assure task info creation is fast // Note: the stats my be internally inconsistent - return new LocalExchangeBufferInfo(bufferedBytes.get(), buffer.size()); + return new LocalExchangeBufferInfo(bufferedBytes.get(), bufferedPages.get()); } void addPage(PageReference pageReference) @@ -75,6 +79,7 @@ void addPage(PageReference pageReference) // buffered bytes must be updated before adding to the buffer to assure // the count does not go negative bufferedBytes.addAndGet(retainedSizeInBytes); + bufferedPages.incrementAndGet(); buffer.add(pageReference); added = true; } @@ -125,7 +130,10 @@ public Page removePage() // NOTE: there is no need to acquire a lock here. The buffer is concurrent // and buffered bytes is not expected to be consistent with the buffer (only // best effort). - PageReference pageReference = buffer.poll(); + PageReference pageReference; + synchronized (this) { + pageReference = buffer.poll(); + } if (pageReference == null) { return null; } @@ -133,6 +141,7 @@ public Page removePage() // dereference the page outside of lock, since may trigger a callback Page page = pageReference.removePage(); bufferedBytes.addAndGet(-pageReference.getRetainedSizeInBytes()); + bufferedPages.decrementAndGet(); checkFinished(); @@ -143,13 +152,13 @@ public ListenableFuture waitForReading() { assertNotHoldsLock(); // Fast path, definitely not blocked - if (finishing || !buffer.isEmpty()) { + if (finishing || bufferedPages.get() > 0) { return NOT_BLOCKED; } synchronized (this) { // re-check after synchronizing - if (finishing || !buffer.isEmpty()) { + if (finishing || bufferedPages.get() > 0) { return NOT_BLOCKED; } // if we need to block readers, and the current future is complete, create a new one @@ -168,7 +177,7 @@ public boolean isFinished() } synchronized (this) { // Synchronize to ensure effects of an in-flight close() or finish() are observed - return finishing && buffer.isEmpty(); + return finishing && bufferedPages.get() == 0; } } @@ -200,13 +209,15 @@ public void close() { assertNotHoldsLock(); - List remainingPages = new ArrayList<>(); + List remainingPages; SettableFuture notEmptyFuture; synchronized (this) { finishing = true; - buffer.drainTo(remainingPages); + remainingPages = new ArrayList<>(buffer); + buffer.clear(); bufferedBytes.addAndGet(-remainingPages.stream().mapToLong(PageReference::getRetainedSizeInBytes).sum()); + bufferedPages.addAndGet(-remainingPages.size()); notEmptyFuture = this.notEmptyFuture; this.notEmptyFuture = null; From ab6cc6bb898f9f7712f10f6d3dc8904b14265f85 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 1 Nov 2022 15:32:52 -0400 Subject: [PATCH 4/5] Optimize local exchange Avoid creating PageReference objects for every page --- .../operator/exchange/LocalExchange.java | 72 +++++++++++------ .../exchange/LocalExchangeSource.java | 42 +++++----- .../operator/exchange/PageReference.java | 81 ------------------- .../exchange/PartitioningExchanger.java | 18 ++--- .../exchange/PassthroughExchanger.java | 22 +---- .../operator/exchange/RandomExchanger.java | 12 +-- .../exchange/ScaleWriterExchanger.java | 14 ++-- .../ScaleWriterPartitioningExchanger.java | 15 ++-- .../operator/exchange/TestLocalExchange.java | 10 ++- 9 files changed, 102 insertions(+), 184 deletions(-) delete mode 100644 core/trino-main/src/main/java/io/trino/operator/exchange/PageReference.java diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java index e9e9236b515f..5915f30399e9 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java @@ -13,6 +13,7 @@ */ package io.trino.operator.exchange; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; import io.airlift.slice.XxHash64; @@ -38,14 +39,15 @@ import java.io.Closeable; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.IntStream; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; @@ -70,8 +72,6 @@ public class LocalExchange private final List sources; - private final LocalExchangeMemoryManager memoryManager; - // Physical written bytes for each writer in the same order as source buffers private final List> physicalWrittenBytesSuppliers = new CopyOnWriteArrayList<>(); @@ -102,31 +102,36 @@ public LocalExchange( BlockTypeOperators blockTypeOperators, DataSize writerMinSize) { - ImmutableList.Builder sources = ImmutableList.builder(); int bufferCount = computeBufferCount(partitioning, defaultConcurrency, partitionChannels); - for (int i = 0; i < bufferCount; i++) { - sources.add(new LocalExchangeSource(source -> checkAllSourcesFinished())); - } - this.sources = sources.build(); - - List> buffers = this.sources.stream() - .map(buffer -> (Consumer) buffer::addPage) - .collect(toImmutableList()); - this.memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); if (partitioning.equals(SINGLE_DISTRIBUTION) || partitioning.equals(FIXED_ARBITRARY_DISTRIBUTION)) { - exchangerSupplier = () -> new RandomExchanger(buffers, memoryManager); + LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); + sources = IntStream.range(0, bufferCount) + .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) + .collect(toImmutableList()); + exchangerSupplier = () -> new RandomExchanger(asPageConsumers(sources), memoryManager); } else if (partitioning.equals(FIXED_PASSTHROUGH_DISTRIBUTION)) { - Iterator sourceIterator = this.sources.iterator(); + List memoryManagers = IntStream.range(0, bufferCount) + .mapToObj(i -> new LocalExchangeMemoryManager(maxBufferedBytes.toBytes() / bufferCount)) + .collect(toImmutableList()); + sources = memoryManagers.stream() + .map(memoryManager -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) + .collect(toImmutableList()); + AtomicInteger nextSource = new AtomicInteger(); exchangerSupplier = () -> { - checkState(sourceIterator.hasNext(), "no more sources"); - return new PassthroughExchanger(sourceIterator.next(), maxBufferedBytes.toBytes() / bufferCount, memoryManager::updateMemoryUsage); + int currentSource = nextSource.getAndIncrement(); + checkState(currentSource < sources.size(), "no more sources"); + return new PassthroughExchanger(sources.get(currentSource), memoryManagers.get(currentSource)); }; } else if (partitioning.equals(SCALED_WRITER_ROUND_ROBIN_DISTRIBUTION)) { + LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); + sources = IntStream.range(0, bufferCount) + .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) + .collect(toImmutableList()); exchangerSupplier = () -> new ScaleWriterExchanger( - buffers, + asPageConsumers(sources), memoryManager, maxBufferedBytes.toBytes(), () -> { @@ -149,6 +154,11 @@ else if (isScaledWriterHashDistribution(partitioning)) { bufferCount, writerMinSize.toBytes()); + LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); + sources = IntStream.range(0, bufferCount) + .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) + .collect(toImmutableList()); + exchangerSupplier = () -> { PartitionFunction partitionFunction = createPartitionFunction( nodePartitioningManager, @@ -160,7 +170,7 @@ else if (isScaledWriterHashDistribution(partitioning)) { partitionChannelTypes, partitionHashChannel); ScaleWriterPartitioningExchanger exchanger = new ScaleWriterPartitioningExchanger( - buffers, + asPageConsumers(sources), memoryManager, maxBufferedBytes.toBytes(), createPartitionPagePreparer(partitioning, partitionChannels), @@ -173,6 +183,10 @@ else if (isScaledWriterHashDistribution(partitioning)) { } else if (partitioning.equals(FIXED_HASH_DISTRIBUTION) || partitioning.getCatalogHandle().isPresent() || (partitioning.getConnectorHandle() instanceof MergePartitioningHandle)) { + LocalExchangeMemoryManager memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); + sources = IntStream.range(0, bufferCount) + .mapToObj(i -> new LocalExchangeSource(memoryManager, source -> checkAllSourcesFinished())) + .collect(toImmutableList()); exchangerSupplier = () -> { PartitionFunction partitionFunction = createPartitionFunction( nodePartitioningManager, @@ -184,7 +198,7 @@ else if (partitioning.equals(FIXED_HASH_DISTRIBUTION) || partitioning.getCatalog partitionChannelTypes, partitionHashChannel); return new PartitioningExchanger( - buffers, + asPageConsumers(sources), memoryManager, createPartitionPagePreparer(partitioning, partitionChannels), partitionFunction); @@ -200,11 +214,6 @@ public int getBufferCount() return sources.size(); } - public long getBufferedBytes() - { - return memoryManager.getBufferedBytes(); - } - public synchronized LocalExchangeSinkFactory createSinkFactory() { checkState(!noMoreSinkFactories, "No more sink factories already set"); @@ -388,6 +397,12 @@ private void checkAllSinksComplete() sources.forEach(LocalExchangeSource::finish); } + @VisibleForTesting + LocalExchangeSource getSource(int partitionIndex) + { + return sources.get(partitionIndex); + } + private static void checkNotHoldsLock(Object lock) { checkState(!Thread.holdsLock(lock), "Cannot execute this method while holding a lock"); @@ -434,6 +449,13 @@ else if (partitioning.equals(FIXED_HASH_DISTRIBUTION) || partitioning.getCatalog return bufferCount; } + private static List> asPageConsumers(List sources) + { + return sources.stream() + .map(buffer -> (Consumer) buffer::addPage) + .collect(toImmutableList()); + } + // Sink factory is entirely a pass thought to LocalExchange. // This class only exists as a separate entity to deal with the complex lifecycle caused // by operator factories (e.g., duplicate and noMoreSinkFactories). diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java index ba5331839d05..d5738a930b4f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSource.java @@ -24,8 +24,6 @@ import javax.annotation.concurrent.ThreadSafe; import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.List; import java.util.Queue; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -40,10 +38,11 @@ public class LocalExchangeSource { private static final ListenableFuture NOT_BLOCKED = immediateVoidFuture(); + private final LocalExchangeMemoryManager memoryManager; private final Consumer onFinish; @GuardedBy("this") - private final Queue buffer = new ArrayDeque<>(); + private final Queue buffer = new ArrayDeque<>(); private final AtomicLong bufferedBytes = new AtomicLong(); private final AtomicInteger bufferedPages = new AtomicInteger(); @@ -54,8 +53,9 @@ public class LocalExchangeSource private volatile boolean finishing; - public LocalExchangeSource(Consumer onFinish) + public LocalExchangeSource(LocalExchangeMemoryManager memoryManager, Consumer onFinish) { + this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); this.onFinish = requireNonNull(onFinish, "onFinish is null"); } @@ -66,13 +66,13 @@ public LocalExchangeBufferInfo getBufferInfo() return new LocalExchangeBufferInfo(bufferedBytes.get(), bufferedPages.get()); } - void addPage(PageReference pageReference) + void addPage(Page page) { assertNotHoldsLock(); boolean added = false; SettableFuture notEmptyFuture = null; - long retainedSizeInBytes = pageReference.getRetainedSizeInBytes(); + long retainedSizeInBytes = page.getRetainedSizeInBytes(); synchronized (this) { // ignore pages after finish if (!finishing) { @@ -80,7 +80,7 @@ void addPage(PageReference pageReference) // the count does not go negative bufferedBytes.addAndGet(retainedSizeInBytes); bufferedPages.incrementAndGet(); - buffer.add(pageReference); + buffer.add(page); added = true; } @@ -92,8 +92,7 @@ void addPage(PageReference pageReference) } if (!added) { - // dereference the page outside of lock - pageReference.removePage(); + memoryManager.updateMemoryUsage(-retainedSizeInBytes); } // notify readers outside of lock since this may result in a callback @@ -130,17 +129,18 @@ public Page removePage() // NOTE: there is no need to acquire a lock here. The buffer is concurrent // and buffered bytes is not expected to be consistent with the buffer (only // best effort). - PageReference pageReference; + Page page; synchronized (this) { - pageReference = buffer.poll(); + page = buffer.poll(); } - if (pageReference == null) { + if (page == null) { return null; } // dereference the page outside of lock, since may trigger a callback - Page page = pageReference.removePage(); - bufferedBytes.addAndGet(-pageReference.getRetainedSizeInBytes()); + long retainedSizeInBytes = page.getRetainedSizeInBytes(); + memoryManager.updateMemoryUsage(-retainedSizeInBytes); + bufferedBytes.addAndGet(-retainedSizeInBytes); bufferedPages.decrementAndGet(); checkFinished(); @@ -209,22 +209,26 @@ public void close() { assertNotHoldsLock(); - List remainingPages; + int remainingPagesCount = 0; + long remainingPagesRetainedSizeInBytes = 0; SettableFuture notEmptyFuture; synchronized (this) { finishing = true; - remainingPages = new ArrayList<>(buffer); + for (Page page : buffer) { + remainingPagesCount++; + remainingPagesRetainedSizeInBytes += page.getRetainedSizeInBytes(); + } buffer.clear(); - bufferedBytes.addAndGet(-remainingPages.stream().mapToLong(PageReference::getRetainedSizeInBytes).sum()); - bufferedPages.addAndGet(-remainingPages.size()); + bufferedBytes.addAndGet(-remainingPagesRetainedSizeInBytes); + bufferedPages.addAndGet(-remainingPagesCount); notEmptyFuture = this.notEmptyFuture; this.notEmptyFuture = null; } // free all the remaining pages - remainingPages.forEach(PageReference::removePage); + memoryManager.updateMemoryUsage(-remainingPagesRetainedSizeInBytes); // notify readers outside of lock since this may result in a callback if (notEmptyFuture != null) { diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/PageReference.java b/core/trino-main/src/main/java/io/trino/operator/exchange/PageReference.java deleted file mode 100644 index 8748af0cab70..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/PageReference.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.operator.exchange; - -import io.trino.spi.Page; - -import javax.annotation.concurrent.ThreadSafe; - -import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; - -import static com.google.common.base.MoreObjects.toStringHelper; -import static com.google.common.base.Preconditions.checkArgument; -import static java.util.Objects.requireNonNull; - -@ThreadSafe -final class PageReference -{ - private static final AtomicIntegerFieldUpdater REFERENCE_COUNT_UPDATER = AtomicIntegerFieldUpdater.newUpdater(PageReference.class, "referenceCount"); - - private volatile int referenceCount; - private final Page page; - private final PageReleasedListener onPageReleased; - private final long retainedSizeInBytes; - - public PageReference(Page page, int referenceCount, PageReleasedListener onPageReleased) - { - this.page = requireNonNull(page, "page is null"); - this.onPageReleased = requireNonNull(onPageReleased, "onPageReleased is null"); - checkArgument(referenceCount >= 1, "referenceCount must be at least 1"); - this.referenceCount = referenceCount; - // retained size will be accessed at least once (typically more) so storing the value into this - // field eagerly avoids repeated volatile reads after this point - this.retainedSizeInBytes = page.getRetainedSizeInBytes(); - } - - public long getRetainedSizeInBytes() - { - return retainedSizeInBytes; - } - - public Page removePage() - { - int referenceCount = REFERENCE_COUNT_UPDATER.decrementAndGet(this); - checkArgument(referenceCount >= 0, "Page reference count is negative"); - if (referenceCount == 0) { - onPageReleased.onPageReleased(retainedSizeInBytes); - } - return page; - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("size", retainedSizeInBytes) - .add("referenceCount", referenceCount) - .toString(); - } - - interface PageReleasedListener - { - void onPageReleased(long releasedSizeInBytes); - - static PageReleasedListener forLocalExchangeMemoryManager(LocalExchangeMemoryManager memoryManager) - { - requireNonNull(memoryManager, "memoryManager is null"); - return releasedSizeInBytes -> memoryManager.updateMemoryUsage(-releasedSizeInBytes); - } - } -} diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java index 9985592e3f46..5d810f5bced6 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; import io.trino.operator.PartitionFunction; -import io.trino.operator.exchange.PageReference.PageReleasedListener; import io.trino.spi.Page; import it.unimi.dsi.fastutil.ints.IntArrayList; @@ -32,23 +31,21 @@ class PartitioningExchanger implements LocalExchanger { - private final List> buffers; + private final List> buffers; private final LocalExchangeMemoryManager memoryManager; private final Function partitionedPagePreparer; private final PartitionFunction partitionFunction; @GuardedBy("this") private final IntArrayList[] partitionAssignments; - private final PageReleasedListener onPageReleased; public PartitioningExchanger( - List> partitions, + List> partitions, LocalExchangeMemoryManager memoryManager, Function partitionPagePreparer, PartitionFunction partitionFunction) { this.buffers = ImmutableList.copyOf(requireNonNull(partitions, "partitions is null")); this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); - this.onPageReleased = PageReleasedListener.forLocalExchangeMemoryManager(memoryManager); this.partitionedPagePreparer = requireNonNull(partitionPagePreparer, "partitionPagePreparer is null"); this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null"); @@ -61,7 +58,7 @@ public PartitioningExchanger( @Override public void accept(Page page) { - Consumer wholePagePartition = partitionPageOrFindWholePagePartition(page, partitionedPagePreparer.apply(page)); + Consumer wholePagePartition = partitionPageOrFindWholePagePartition(page, partitionedPagePreparer.apply(page)); if (wholePagePartition != null) { // whole input page will go to this partition, compact the input page avoid over-retaining memory and to // match the behavior of sub-partitioned pages that copy positions out @@ -71,7 +68,7 @@ public void accept(Page page) } @Nullable - private synchronized Consumer partitionPageOrFindWholePagePartition(Page page, Page partitionPage) + private synchronized Consumer partitionPageOrFindWholePagePartition(Page page, Page partitionPage) { // assign each row to a partition. The assignments lists are all expected to cleared by the previous iterations for (int position = 0; position < partitionPage.getPositionCount(); position++) { @@ -104,11 +101,10 @@ private synchronized Consumer partitionPageOrFindWholePagePartiti } // This is safe to call without synchronizing because the partition buffers are themselves threadsafe - private void sendPageToPartition(Consumer buffer, Page pageSplit) + private void sendPageToPartition(Consumer buffer, Page pageSplit) { - PageReference pageReference = new PageReference(pageSplit, 1, onPageReleased); - memoryManager.updateMemoryUsage(pageReference.getRetainedSizeInBytes()); - buffer.accept(pageReference); + memoryManager.updateMemoryUsage(pageSplit.getRetainedSizeInBytes()); + buffer.accept(pageSplit); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/PassthroughExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/PassthroughExchanger.java index cbf15e688880..38a645efa309 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/PassthroughExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/PassthroughExchanger.java @@ -14,11 +14,8 @@ package io.trino.operator.exchange; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.operator.exchange.PageReference.PageReleasedListener; import io.trino.spi.Page; -import java.util.function.LongConsumer; - import static java.util.Objects.requireNonNull; public class PassthroughExchanger @@ -26,29 +23,18 @@ public class PassthroughExchanger { private final LocalExchangeSource localExchangeSource; private final LocalExchangeMemoryManager bufferMemoryManager; - private final LongConsumer memoryTracker; - private final PageReleasedListener onPageReleased; - public PassthroughExchanger(LocalExchangeSource localExchangeSource, long bufferMaxMemory, LongConsumer memoryTracker) + public PassthroughExchanger(LocalExchangeSource localExchangeSource, LocalExchangeMemoryManager bufferMemoryManager) { this.localExchangeSource = requireNonNull(localExchangeSource, "localExchangeSource is null"); - this.memoryTracker = requireNonNull(memoryTracker, "memoryTracker is null"); - bufferMemoryManager = new LocalExchangeMemoryManager(bufferMaxMemory); - onPageReleased = releasedSizeInBytes -> { - this.bufferMemoryManager.updateMemoryUsage(-releasedSizeInBytes); - this.memoryTracker.accept(-releasedSizeInBytes); - }; + this.bufferMemoryManager = requireNonNull(bufferMemoryManager, "bufferMemoryManager is null"); } @Override public void accept(Page page) { - PageReference pageReference = new PageReference(page, 1, onPageReleased); - long retainedSizeInBytes = pageReference.getRetainedSizeInBytes(); - bufferMemoryManager.updateMemoryUsage(retainedSizeInBytes); - memoryTracker.accept(retainedSizeInBytes); - - localExchangeSource.addPage(pageReference); + bufferMemoryManager.updateMemoryUsage(page.getRetainedSizeInBytes()); + localExchangeSource.addPage(page); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/RandomExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/RandomExchanger.java index 8c1eadfb1832..e7a354346628 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/RandomExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/RandomExchanger.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.operator.exchange.PageReference.PageReleasedListener; import io.trino.spi.Page; import java.util.List; @@ -27,24 +26,21 @@ class RandomExchanger implements LocalExchanger { - private final List> buffers; + private final List> buffers; private final LocalExchangeMemoryManager memoryManager; - private final PageReleasedListener onPageReleased; - public RandomExchanger(List> buffers, LocalExchangeMemoryManager memoryManager) + public RandomExchanger(List> buffers, LocalExchangeMemoryManager memoryManager) { this.buffers = ImmutableList.copyOf(requireNonNull(buffers, "buffers is null")); this.memoryManager = requireNonNull(memoryManager, "memoryManager is null"); - this.onPageReleased = PageReleasedListener.forLocalExchangeMemoryManager(memoryManager); } @Override public void accept(Page page) { - PageReference pageReference = new PageReference(page, 1, onPageReleased); - memoryManager.updateMemoryUsage(pageReference.getRetainedSizeInBytes()); + memoryManager.updateMemoryUsage(page.getRetainedSizeInBytes()); int randomIndex = ThreadLocalRandom.current().nextInt(buffers.size()); - buffers.get(randomIndex).accept(pageReference); + buffers.get(randomIndex).accept(page); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java index 973277d4ea62..2b217790e7e7 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterExchanger.java @@ -17,7 +17,6 @@ import com.google.common.util.concurrent.ListenableFuture; import io.airlift.log.Logger; import io.airlift.units.DataSize; -import io.trino.operator.exchange.PageReference.PageReleasedListener; import io.trino.spi.Page; import java.util.List; @@ -35,12 +34,11 @@ public class ScaleWriterExchanger { private static final Logger log = Logger.get(ScaleWriterExchanger.class); - private final List> buffers; + private final List> buffers; private final LocalExchangeMemoryManager memoryManager; private final long maxBufferedBytes; private final Supplier physicalWrittenBytesSupplier; private final long writerMinSize; - private final PageReleasedListener onPageReleased; // Start with single writer and increase the writer count based on // physical written bytes and buffer utilization. @@ -49,7 +47,7 @@ public class ScaleWriterExchanger private int nextWriterIndex = -1; public ScaleWriterExchanger( - List> buffers, + List> buffers, LocalExchangeMemoryManager memoryManager, long maxBufferedBytes, Supplier physicalWrittenBytesSupplier, @@ -60,16 +58,14 @@ public ScaleWriterExchanger( this.maxBufferedBytes = maxBufferedBytes; this.physicalWrittenBytesSupplier = requireNonNull(physicalWrittenBytesSupplier, "physicalWrittenBytesSupplier is null"); this.writerMinSize = writerMinSize.toBytes(); - this.onPageReleased = PageReleasedListener.forLocalExchangeMemoryManager(memoryManager); } @Override public void accept(Page page) { - Consumer buffer = buffers.get(getNextWriterIndex()); - PageReference pageReference = new PageReference(page, 1, onPageReleased); - memoryManager.updateMemoryUsage(pageReference.getRetainedSizeInBytes()); - buffer.accept(pageReference); + Consumer buffer = buffers.get(getNextWriterIndex()); + memoryManager.updateMemoryUsage(page.getRetainedSizeInBytes()); + buffer.accept(page); } private int getNextWriterIndex() diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java index ca92e425bb07..e8d193661a5f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/ScaleWriterPartitioningExchanger.java @@ -16,7 +16,6 @@ import com.google.common.util.concurrent.ListenableFuture; import io.trino.operator.PartitionFunction; -import io.trino.operator.exchange.PageReference.PageReleasedListener; import io.trino.spi.Page; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.longs.Long2IntMap; @@ -30,7 +29,6 @@ import java.util.function.Consumer; import java.util.function.Function; -import static io.trino.operator.exchange.PageReference.PageReleasedListener.forLocalExchangeMemoryManager; import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId; import static io.trino.operator.exchange.UniformPartitionRebalancer.WriterPartitionId.serialize; import static java.util.Arrays.fill; @@ -39,13 +37,12 @@ public class ScaleWriterPartitioningExchanger implements LocalExchanger { - private final List> buffers; + private final List> buffers; private final LocalExchangeMemoryManager memoryManager; private final long maxBufferedBytes; private final Function partitionedPagePreparer; private final PartitionFunction partitionFunction; private final UniformPartitionRebalancer partitionRebalancer; - private final PageReleasedListener onPageReleased; private final IntArrayList[] writerAssignments; private final int[] partitionRowCounts; @@ -64,7 +61,7 @@ public class ScaleWriterPartitioningExchanger private final Long2LongMap writerPartitionRowCounts = new Long2LongOpenHashMap(); public ScaleWriterPartitioningExchanger( - List> buffers, + List> buffers, LocalExchangeMemoryManager memoryManager, long maxBufferedBytes, Function partitionedPagePreparer, @@ -78,7 +75,6 @@ public ScaleWriterPartitioningExchanger( this.partitionedPagePreparer = requireNonNull(partitionedPagePreparer, "partitionedPagePreparer is null"); this.partitionFunction = requireNonNull(partitionFunction, "partitionFunction is null"); this.partitionRebalancer = requireNonNull(partitionRebalancer, "partitionRebalancer is null"); - this.onPageReleased = forLocalExchangeMemoryManager(memoryManager); // Initialize writerAssignments with the buffer size writerAssignments = new IntArrayList[buffers.size()]; @@ -190,10 +186,9 @@ private int getNextWriterId(int partitionId) return partitionRebalancer.getWriterId(partitionId, partitionWriterIndexes[partitionId]++); } - private void sendPageToPartition(Consumer buffer, Page pageSplit) + private void sendPageToPartition(Consumer buffer, Page pageSplit) { - PageReference pageReference = new PageReference(pageSplit, 1, onPageReleased); - memoryManager.updateMemoryUsage(pageReference.getRetainedSizeInBytes()); - buffer.accept(pageReference); + memoryManager.updateMemoryUsage(pageSplit.getRetainedSizeInBytes()); + buffer.accept(pageSplit); } } diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index 5561c01417eb..3a19b4e0f732 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -783,13 +783,13 @@ public void testPartition() assertSource(sourceA, 1); assertSource(sourceB, 1); - assertTrue(exchange.getBufferedBytes() >= retainedSizeOfPages(1)); + assertTrue(sourceA.getBufferInfo().getBufferedBytes() + sourceB.getBufferInfo().getBufferedBytes() >= retainedSizeOfPages(1)); sink.addPage(createPage(0)); assertSource(sourceA, 2); assertSource(sourceB, 2); - assertTrue(exchange.getBufferedBytes() >= retainedSizeOfPages(2)); + assertTrue(sourceA.getBufferInfo().getBufferedBytes() + sourceB.getBufferInfo().getBufferedBytes() >= retainedSizeOfPages(2)); assertPartitionedRemovePage(sourceA, 0, 2); assertSource(sourceA, 1); @@ -1155,7 +1155,11 @@ private static void assertSinkFinished(LocalExchangeSink sink) private static void assertExchangeTotalBufferedBytes(LocalExchange exchange, int pageCount) { - assertEquals(exchange.getBufferedBytes(), retainedSizeOfPages(pageCount)); + long bufferedBytes = 0; + for (int i = 0; i < exchange.getBufferCount(); i++) { + bufferedBytes += exchange.getSource(i).getBufferInfo().getBufferedBytes(); + } + assertEquals(bufferedBytes, retainedSizeOfPages(pageCount)); } private static Page createPage(int i) From af5cc04dbd20c3228345307eabadd43e0ecd3e72 Mon Sep 17 00:00:00 2001 From: Andrii Rosa Date: Tue, 1 Nov 2022 15:37:30 -0400 Subject: [PATCH 5/5] Drop unnecessary synchronization A new instance is created for each ExchangeSink --- .../io/trino/operator/exchange/PartitioningExchanger.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java b/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java index 5d810f5bced6..62cee860defe 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/PartitioningExchanger.java @@ -20,7 +20,7 @@ import it.unimi.dsi.fastutil.ints.IntArrayList; import javax.annotation.Nullable; -import javax.annotation.concurrent.GuardedBy; +import javax.annotation.concurrent.NotThreadSafe; import java.util.List; import java.util.function.Consumer; @@ -28,6 +28,7 @@ import static java.util.Objects.requireNonNull; +@NotThreadSafe class PartitioningExchanger implements LocalExchanger { @@ -35,7 +36,6 @@ class PartitioningExchanger private final LocalExchangeMemoryManager memoryManager; private final Function partitionedPagePreparer; private final PartitionFunction partitionFunction; - @GuardedBy("this") private final IntArrayList[] partitionAssignments; public PartitioningExchanger( @@ -68,7 +68,7 @@ public void accept(Page page) } @Nullable - private synchronized Consumer partitionPageOrFindWholePagePartition(Page page, Page partitionPage) + private Consumer partitionPageOrFindWholePagePartition(Page page, Page partitionPage) { // assign each row to a partition. The assignments lists are all expected to cleared by the previous iterations for (int position = 0; position < partitionPage.getPositionCount(); position++) {