From 1fded6e2ebd71a875e39c4c8318c517843bd7783 Mon Sep 17 00:00:00 2001 From: Alex Barreto Date: Mon, 11 Jan 2021 08:29:48 -0600 Subject: [PATCH] [SPARK-33084][CORE][SQL] Rename Unit test file and use fake ivy link ### What changes were proposed in this pull request? According to https://github.com/apache/spark/pull/29966#discussion_r554514344 Use wrong name about suite file, this pr to fix this problem. And change to use some fake ivy link for this test ### Why are the changes needed? Follow file name rule ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? No Closes #31118 from AngersZhuuuu/SPARK-33084-FOLLOW-UP. Authored-by: angerszhu Signed-off-by: HyukjinKwon --- R/pkg/DESCRIPTION | 2 +- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/SQLContext.R | 2 +- R/pkg/R/install.R | 6 +- R/pkg/R/mllib_classification.R | 4 +- R/pkg/R/mllib_clustering.R | 4 +- R/pkg/R/mllib_recommendation.R | 2 +- R/pkg/R/mllib_regression.R | 2 +- R/pkg/R/mllib_stat.R | 2 +- R/pkg/R/mllib_tree.R | 12 +- R/pkg/R/stats.R | 3 +- R/pkg/inst/worker/worker.R | 9 +- R/pkg/vignettes/sparkr-vignettes.Rmd | 2 +- .../client/TransportClientFactory.java | 7 +- .../spark/network/util/TransportConf.java | 13 +- .../sort/BypassMergeSortShuffleWriter.java | 5 +- .../shuffle/sort/UnsafeShuffleWriter.java | 7 +- .../unsafe/sort/UnsafeExternalSorter.java | 6 +- .../spark/ExecutorAllocationManager.scala | 6 +- .../spark/api/python/PythonRunner.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 3 +- .../spark/internal/config/package.scala | 29 + .../spark/shuffle/ShuffleBlockPusher.scala | 450 +++++++++++ .../spark/shuffle/ShuffleWriteProcessor.scala | 19 +- .../apache/spark/shuffle/ShuffleWriter.scala | 3 + .../shuffle/sort/SortShuffleWriter.scala | 6 +- .../spark/status/AppStatusListener.scala | 9 +- .../org/apache/spark/storage/BlockId.scala | 11 +- .../executor_list_json_expectation.json | 22 + .../executor_memory_usage_expectation.json | 92 ++- ...tor_node_excludeOnFailure_expectation.json | 92 ++- ...ludeOnFailure_unexcluding_expectation.json | 88 +++ .../org/apache/spark/SparkContextSuite.scala | 16 +- .../shuffle/ShuffleBlockPusherSuite.scala | 355 +++++++++ .../spark/util/DependencyUtilsSuite.scala | 60 ++ dev/deps/spark-deps-hadoop-2.7-hive-2.3 | 2 +- dev/deps/spark-deps-hadoop-3.2-hive-2.3 | 2 +- docs/_layouts/global.html | 1 + docs/css/main.css | 2 +- docs/index.md | 2 + docs/sql-data-sources-hive-tables.md | 27 +- docs/sql-data-sources-orc.md | 135 +++- docs/sql-migration-guide.md | 2 +- docs/sql-ref-ansi-compliance.md | 1 + docs/sql-ref-syntax-qry-select-like.md | 60 +- docs/sql-ref-syntax-qry-select.md | 14 +- .../structured-streaming-kafka-integration.md | 17 + .../spark/sql/avro/AvroLogicalTypeSuite.scala | 2 +- .../spark/kafka010/KafkaTokenSparkConf.scala | 6 + .../spark/kafka010/KafkaTokenUtil.scala | 10 +- .../kafka010/KafkaDelegationTokenTest.scala | 4 + .../kafka010/KafkaTokenSparkConfSuite.scala | 10 + .../spark/kafka010/KafkaTokenUtilSuite.scala | 6 + .../classification/NaiveBayesSuite.scala | 4 +- pom.xml | 11 +- project/SparkBuild.scala | 30 +- python/docs/source/getting_started/index.rst | 3 + .../docs/source/getting_started/install.rst | 10 +- python/docs/source/migration_guide/index.rst | 12 +- python/docs/source/reference/pyspark.ml.rst | 12 +- .../docs/source/reference/pyspark.mllib.rst | 4 +- python/docs/source/user_guide/index.rst | 12 + python/pyspark/find_spark_home.py | 2 +- python/pyspark/sql/functions.py | 16 +- python/pyspark/sql/functions.pyi | 6 +- python/setup.py | 14 +- .../org/apache/spark/deploy/k8s/Config.scala | 8 + .../k8s/submit/KubernetesClientUtils.scala | 80 +- .../cluster/k8s/ExecutorPodsSnapshot.scala | 26 +- .../spark/deploy/k8s/submit/ClientSuite.scala | 21 +- .../submit/KubernetesClientUtilsSuite.scala | 79 ++ .../k8s/ExecutorLifecycleTestUtils.scala | 3 + .../mesos/MesosSchedulerBackendUtil.scala | 2 +- .../spark/deploy/yarn/YarnAllocator.scala | 2 +- .../spark/sql/QueryCompilationErrors.scala | 170 +++- .../spark/sql/QueryExecutionErrors.scala | 56 ++ .../sql/catalyst/analysis/Analyzer.scala | 17 +- .../catalyst/analysis/v2ResolutionPlans.scala | 2 +- .../catalog/ExternalCatalogUtils.scala | 10 + .../catalog/GlobalTempViewManager.scala | 5 +- .../catalyst/catalog/InMemoryCatalog.scala | 49 +- .../sql/catalyst/catalog/SessionCatalog.scala | 81 +- .../catalyst/catalog/functionResources.scala | 4 +- .../sql/catalyst/catalog/interface.scala | 74 +- .../expressions/datetimeExpressions.scala | 72 +- .../expressions/objects/objects.scala | 2 +- .../expressions/regexpExpressions.scala | 6 +- .../expressions/windowExpressions.scala | 2 +- .../optimizer/CostBasedJoinReorder.scala | 13 +- .../sql/catalyst/optimizer/Optimizer.scala | 14 + .../SimplifyConditionalsInPredicate.scala | 6 - .../sql/catalyst/optimizer/expressions.scala | 81 +- .../sql/catalyst/parser/AstBuilder.scala | 68 +- .../plans/logical/AnalysisHelper.scala | 9 + .../plans/logical/basicLogicalOperators.scala | 13 +- .../BasicStatsPlanVisitor.scala | 10 +- .../sql/catalyst/util/DateTimeUtils.scala | 6 +- .../AnalysisExceptionPositionSuite.scala | 25 + .../analysis/UnsupportedOperationsSuite.scala | 4 +- .../expressions/DateExpressionsSuite.scala | 34 +- .../optimizer/LikeSimplificationSuite.scala | 68 ++ ...SimplifyConditionalsInPredicateSuite.scala | 11 +- .../joinReorder/JoinReorderSuite.scala | 15 + .../StarJoinCostBasedReorderSuite.scala | 8 +- .../sql/catalyst/parser/DDLParserSuite.scala | 82 +- .../catalyst/parser/ParserUtilsSuite.scala | 4 +- .../BasicStatsEstimationSuite.scala | 11 + .../catalyst/util/DateTimeUtilsSuite.scala | 7 + .../spark/sql/connector/InMemoryTable.scala | 10 +- .../sql/execution/UnsafeKVExternalSorter.java | 3 +- .../BaseScriptTransformationExec.scala | 2 +- .../spark/sql/execution/CacheManager.scala | 2 +- .../spark/sql/execution/command/ddl.scala | 2 + .../sql/execution/datasources/rules.scala | 3 +- .../v2/AlterTableRenamePartitionExec.scala | 7 +- .../datasources/v2/DataSourceV2Strategy.scala | 26 +- .../datasources/v2/DescribeColumnExec.scala | 11 +- .../v2/DescribeNamespaceExec.scala | 12 +- .../datasources/v2/DescribeTableExec.scala | 13 +- .../datasources/v2/ReplaceTableExec.scala | 14 +- .../v2/SetCatalogAndNamespaceExec.scala | 4 +- .../v2/ShowCurrentNamespaceExec.scala | 10 +- .../datasources/v2/ShowNamespacesExec.scala | 7 +- .../v2/ShowTablePropertiesExec.scala | 10 +- .../datasources/v2/ShowTablesExec.scala | 9 +- .../datasources/v2/V2CommandExec.scala | 11 +- .../v2/WriteToDataSourceV2Exec.scala | 22 +- .../v2/jdbc/JDBCTableCatalog.scala | 4 +- .../execution/streaming/StreamExecution.scala | 9 +- .../resources/sql-tests/inputs/datetime.sql | 7 + .../resources/sql-tests/inputs/group-by.sql | 10 + .../sql-tests/results/ansi/datetime.sql.out | 44 +- .../sql-tests/results/datetime-legacy.sql.out | 42 +- .../sql-tests/results/datetime.sql.out | 42 +- .../sql-tests/results/group-by.sql.out | 63 +- .../sql-tests/results/show_columns.sql.out | 8 +- sql/core/src/test/resources/test_script.py | 2 +- .../approved-plans-v1_4/q13.sf100/explain.txt | 132 ++-- .../q13.sf100/simplified.txt | 34 +- .../approved-plans-v1_4/q17.sf100/explain.txt | 194 ++--- .../q17.sf100/simplified.txt | 130 ++-- .../approved-plans-v1_4/q18.sf100/explain.txt | 158 ++-- .../q18.sf100/simplified.txt | 50 +- .../approved-plans-v1_4/q19.sf100/explain.txt | 368 ++++----- .../q19.sf100/simplified.txt | 116 +-- .../approved-plans-v1_4/q2.sf100/explain.txt | 128 ++- .../q2.sf100/simplified.txt | 98 ++- .../q24a.sf100/explain.txt | 118 +-- .../q24a.sf100/simplified.txt | 34 +- .../q24b.sf100/explain.txt | 118 +-- .../q24b.sf100/simplified.txt | 34 +- .../approved-plans-v1_4/q25.sf100/explain.txt | 194 ++--- .../q25.sf100/simplified.txt | 130 ++-- .../approved-plans-v1_4/q33.sf100/explain.txt | 264 +++---- .../q33.sf100/simplified.txt | 58 +- .../approved-plans-v1_4/q5.sf100/explain.txt | 220 +++--- .../q5.sf100/simplified.txt | 64 +- .../approved-plans-v1_4/q52.sf100/explain.txt | 138 ++-- .../q52.sf100/simplified.txt | 26 +- .../approved-plans-v1_4/q54.sf100/explain.txt | 726 +++++++++--------- .../q54.sf100/simplified.txt | 244 +++--- .../approved-plans-v1_4/q55.sf100/explain.txt | 134 ++-- .../q55.sf100/simplified.txt | 26 +- .../approved-plans-v1_4/q72.sf100/explain.txt | 264 +++---- .../q72.sf100/simplified.txt | 150 ++-- .../approved-plans-v1_4/q81.sf100/explain.txt | 570 +++++++------- .../q81.sf100/simplified.txt | 142 ++-- .../approved-plans-v1_4/q91.sf100/explain.txt | 306 ++++---- .../q91.sf100/simplified.txt | 62 +- .../q18a.sf100/explain.txt | 306 ++++---- .../q18a.sf100/simplified.txt | 54 +- .../approved-plans-v2_7/q5a.sf100/explain.txt | 210 ++--- .../q5a.sf100/simplified.txt | 64 +- .../approved-plans-v2_7/q72.sf100/explain.txt | 264 +++---- .../q72.sf100/simplified.txt | 150 ++-- .../spark/sql/CharVarcharTestSuite.scala | 23 + .../org/apache/spark/sql/JoinSuite.scala | 123 ++- .../org/apache/spark/sql/SQLQuerySuite.scala | 33 + .../spark/sql/ShowCreateTableSuite.scala | 25 +- .../sql/connector/DataSourceV2SQLSuite.scala | 128 +-- .../BaseScriptTransformationSuite.scala | 5 +- .../UnsafeKVExternalSorterSuite.scala | 95 ++- .../execution/WholeStageCodegenSuite.scala | 41 +- .../PassThroughEncodingSuite.scala | 4 +- .../AlterTableDropPartitionSuiteBase.scala | 11 + .../AlterTableRenamePartitionSuiteBase.scala | 15 + .../sql/execution/command/DDLSuite.scala | 30 - .../command/DropTableParserSuite.scala | 18 +- .../command/ShowNamespacesParserSuite.scala | 70 ++ .../command/ShowNamespacesSuiteBase.scala | 131 ++++ .../v1/AlterTableAddPartitionSuite.scala | 33 +- .../v1/AlterTableDropPartitionSuite.scala | 1 + .../command/v1/ShowNamespacesSuite.scala | 60 ++ .../command/v1/ShowPartitionsSuite.scala | 12 + .../v2/AlterTableDropPartitionSuite.scala | 2 +- .../command/v2/ShowNamespacesSuite.scala | 72 ++ sql/create-docs.sh | 6 +- sql/gen-sql-api-docs.py | 1 + .../hive/thriftserver/SparkSQLCLIDriver.scala | 50 +- .../sql/hive/thriftserver/CliSuite.scala | 23 + .../spark/sql/hive/HiveExternalCatalog.scala | 33 +- .../spark/sql/hive/HiveSessionCatalog.scala | 9 +- .../hive/execution/InsertIntoHiveTable.scala | 2 + .../org/apache/spark/sql/hive/hiveUDFs.scala | 8 +- .../client/HivePartitionFilteringSuite.scala | 71 +- .../sql/hive/execution/SQLQuerySuite.scala | 50 +- .../command/AlterTableAddPartitionSuite.scala | 20 +- .../AlterTableDropPartitionSuite.scala | 51 +- .../AlterTableRenamePartitionSuite.scala | 18 +- .../execution/command/CommandSuiteBase.scala | 8 + .../execution/command/DropTableSuite.scala | 11 +- .../command/ShowNamespacesSuite.scala | 54 ++ .../command/ShowPartitionsSuite.scala | 9 + .../execution/command/ShowTablesSuite.scala | 11 +- 214 files changed, 7112 insertions(+), 4194 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/DependencyUtilsSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 2043336245..c141baa51b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -11,7 +11,7 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) License: Apache License (== 2.0) -URL: https://www.apache.org/ https://spark.apache.org/ +URL: https://www.apache.org https://spark.apache.org BugReports: https://spark.apache.org/contributing.html SystemRequirements: Java (>= 8, < 12) Depends: diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 8ca338f099..72d96151f6 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -880,7 +880,7 @@ setMethod("toJSON", #' Save the contents of SparkDataFrame as a JSON file #' -#' Save the contents of a SparkDataFrame as a JSON file (\href{http://jsonlines.org/}{ +#' Save the contents of a SparkDataFrame as a JSON file (\href{https://jsonlines.org/}{ #' JSON Lines text format or newline-delimited JSON}). Files written out #' with this method can be read back in as a SparkDataFrame using read.json(). #' diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index 5ed0481f33..14262e1a74 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -374,7 +374,7 @@ setMethod("toDF", signature(x = "RDD"), #' Create a SparkDataFrame from a JSON file. #' #' Loads a JSON file, returning the result as a SparkDataFrame -#' By default, (\href{http://jsonlines.org/}{JSON Lines text format or newline-delimited JSON} +#' By default, (\href{https://jsonlines.org/}{JSON Lines text format or newline-delimited JSON} #' ) is supported. For JSON (one record per file), set a named property \code{multiLine} to #' \code{TRUE}. #' It goes through the entire dataset once to determine the schema. diff --git a/R/pkg/R/install.R b/R/pkg/R/install.R index 5bc5ae07c5..bbb9188cd0 100644 --- a/R/pkg/R/install.R +++ b/R/pkg/R/install.R @@ -39,11 +39,11 @@ #' version number in the format of "x.y" where x and y are integer. #' If \code{hadoopVersion = "without"}, "Hadoop free" build is installed. #' See -#' \href{http://spark.apache.org/docs/latest/hadoop-provided.html}{ +#' \href{https://spark.apache.org/docs/latest/hadoop-provided.html}{ #' "Hadoop Free" Build} for more information. #' Other patched version names can also be used, e.g. \code{"cdh4"} #' @param mirrorUrl base URL of the repositories to use. The directory layout should follow -#' \href{http://www.apache.org/dyn/closer.lua/spark/}{Apache mirrors}. +#' \href{https://www.apache.org/dyn/closer.lua/spark/}{Apache mirrors}. #' @param localDir a local directory where Spark is installed. The directory contains #' version-specific folders of Spark packages. Default is path to #' the cache directory: @@ -64,7 +64,7 @@ #'} #' @note install.spark since 2.1.0 #' @seealso See available Hadoop versions: -#' \href{http://spark.apache.org/downloads.html}{Apache Spark} +#' \href{https://spark.apache.org/downloads.html}{Apache Spark} install.spark <- function(hadoopVersion = "2.7", mirrorUrl = NULL, localDir = NULL, overwrite = FALSE) { sparkHome <- Sys.getenv("SPARK_HOME") diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R index ec83b6bd40..71ebe4e26e 100644 --- a/R/pkg/R/mllib_classification.R +++ b/R/pkg/R/mllib_classification.R @@ -425,7 +425,7 @@ setMethod("write.ml", signature(object = "LogisticRegressionModel", path = "char #' predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' Only categorical data is supported. #' For more details, see -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html}{ #' Multilayer Perceptron} #' #' @param data a \code{SparkDataFrame} of observations and labels for model fitting. @@ -574,7 +574,7 @@ setMethod("write.ml", signature(object = "MultilayerPerceptronClassificationMode #' @rdname spark.naiveBayes #' @aliases spark.naiveBayes,SparkDataFrame,formula-method #' @name spark.naiveBayes -#' @seealso e1071: \url{https://cran.r-project.org/package=e1071} +#' @seealso e1071: \url{https://cran.r-project.org/web/packages/e1071/index.html} #' @examples #' \dontrun{ #' data <- as.data.frame(UCBAdmissions) diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index 8bc1535346..ff7cbd8fc9 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -204,7 +204,7 @@ setMethod("write.ml", signature(object = "BisectingKMeansModel", path = "charact #' @return \code{spark.gaussianMixture} returns a fitted multivariate gaussian mixture model. #' @rdname spark.gaussianMixture #' @name spark.gaussianMixture -#' @seealso mixtools: \url{https://cran.r-project.org/package=mixtools} +#' @seealso mixtools: \url{https://cran.r-project.org/web/packages/mixtools/index.html} #' @examples #' \dontrun{ #' sparkR.session() @@ -483,7 +483,7 @@ setMethod("write.ml", signature(object = "KMeansModel", path = "character"), #' @return \code{spark.lda} returns a fitted Latent Dirichlet Allocation model. #' @rdname spark.lda #' @aliases spark.lda,SparkDataFrame-method -#' @seealso topicmodels: \url{https://cran.r-project.org/package=topicmodels} +#' @seealso topicmodels: \url{https://cran.r-project.org/web/packages/topicmodels/index.html} #' @examples #' \dontrun{ #' text <- read.df("data/mllib/sample_lda_libsvm_data.txt", source = "libsvm") diff --git a/R/pkg/R/mllib_recommendation.R b/R/pkg/R/mllib_recommendation.R index d238ff93ed..87a1bc991f 100644 --- a/R/pkg/R/mllib_recommendation.R +++ b/R/pkg/R/mllib_recommendation.R @@ -30,7 +30,7 @@ setClass("ALSModel", representation(jobj = "jobj")) #' to make predictions on new data, and \code{write.ml}/\code{read.ml} to save/load fitted models. #' #' For more details, see -#' \href{http://spark.apache.org/docs/latest/ml-collaborative-filtering.html}{MLlib: +#' \href{https://spark.apache.org/docs/latest/ml-collaborative-filtering.html}{MLlib: #' Collaborative Filtering}. #' #' @param data a SparkDataFrame for training. diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R index b2228a1416..db9f367407 100644 --- a/R/pkg/R/mllib_regression.R +++ b/R/pkg/R/mllib_regression.R @@ -475,7 +475,7 @@ setMethod("write.ml", signature(object = "IsotonicRegressionModel", path = "char #' @param ... additional arguments passed to the method. #' @return \code{spark.survreg} returns a fitted AFT survival regression model. #' @rdname spark.survreg -#' @seealso survival: \url{https://cran.r-project.org/package=survival} +#' @seealso survival: \url{https://cran.r-project.org/web/packages/survival/index.html} #' @examples #' \dontrun{ #' df <- createDataFrame(ovarian) diff --git a/R/pkg/R/mllib_stat.R b/R/pkg/R/mllib_stat.R index 6db4d5d483..f82fb589bb 100644 --- a/R/pkg/R/mllib_stat.R +++ b/R/pkg/R/mllib_stat.R @@ -49,7 +49,7 @@ setClass("KSTest", representation(jobj = "jobj")) #' @rdname spark.kstest #' @aliases spark.kstest,SparkDataFrame-method #' @name spark.kstest -#' @seealso \href{http://spark.apache.org/docs/latest/mllib-statistics.html#hypothesis-testing}{ +#' @seealso \href{https://spark.apache.org/docs/latest/mllib-statistics.html#hypothesis-testing}{ #' MLlib: Hypothesis Testing} #' @examples #' \dontrun{ diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R index b5a014b0a3..f3192ee9b1 100644 --- a/R/pkg/R/mllib_tree.R +++ b/R/pkg/R/mllib_tree.R @@ -127,9 +127,9 @@ print.summary.decisionTree <- function(x) { #' \code{write.ml}/\code{read.ml} to save/load fitted models. #' For more details, see # nolint start -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-regression}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-regression}{ #' GBT Regression} and -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-classifier}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#gradient-boosted-tree-classifier}{ #' GBT Classification} # nolint end #' @@ -343,9 +343,9 @@ setMethod("write.ml", signature(object = "GBTClassificationModel", path = "chara #' save/load fitted models. #' For more details, see # nolint start -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-regression}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-regression}{ #' Random Forest Regression} and -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-classifier}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#random-forest-classifier}{ #' Random Forest Classification} # nolint end #' @@ -568,9 +568,9 @@ setMethod("write.ml", signature(object = "RandomForestClassificationModel", path #' save/load fitted models. #' For more details, see # nolint start -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-regression}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-regression}{ #' Decision Tree Regression} and -#' \href{http://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-classifier}{ +#' \href{https://spark.apache.org/docs/latest/ml-classification-regression.html#decision-tree-classifier}{ #' Decision Tree Classification} # nolint end #' diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 7252351ebe..0aabceef22 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -109,7 +109,8 @@ setMethod("corr", #' #' Finding frequent items for columns, possibly with false positives. #' Using the frequent element count algorithm described in -#' \url{https://doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. +#' \url{https://dl.acm.org/doi/10.1145/762471.762473}, proposed by Karp, Schenker, +#' and Papadimitriou. #' #' @param x A SparkDataFrame. #' @param cols A vector column names to search frequent items in. diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index dd271f91d0..7fc4680bad 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -196,7 +196,7 @@ if (isEmpty != 0) { outputs <- list() for (i in seq_len(length(data))) { # Timing reading input data for execution - inputElap <- elapsedSecs() + computeStart <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], colNames, computeFunc, data[[i]]) computeElap <- elapsedSecs() @@ -204,17 +204,18 @@ if (isEmpty != 0) { outputs[[length(outputs) + 1L]] <- output } else { outputResult(serializer, output, outputCon) + outputComputeElapsDiff <- outputComputeElapsDiff + (elapsedSecs() - computeElap) } - outputElap <- elapsedSecs() - computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) - outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) + computeInputElapsDiff <- computeInputElapsDiff + (computeElap - computeStart) } if (serializer == "arrow") { # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html # rbind.fill might be an alternative to make it faster if plyr is installed. + outputStart <- elapsedSecs() combined <- do.call("rbind", outputs) SparkR:::writeSerializeInArrow(outputCon, combined) + outputComputeElapsDiff <- elapsedSecs() - outputStart } } } else { diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index a060874869..3177b54dc5 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -1007,7 +1007,7 @@ perplexity #### Alternating Least Squares -`spark.als` learns latent factors in [collaborative filtering](https://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) via [alternating least squares](https://dl.acm.org/citation.cfm?id=1608614). +`spark.als` learns latent factors in [collaborative filtering](https://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) via [alternating least squares](https://dl.acm.org/doi/10.1109/MC.2009.263). There are multiple options that can be configured in `spark.als`, including `rank`, `reg`, and `nonnegative`. For a complete list, refer to the help file. diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 24c436a504..43408d43e5 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -254,7 +254,7 @@ TransportClient createClient(InetSocketAddress address) // Disable Nagle's Algorithm since we don't want packets to wait .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionCreationTimeoutMs()) .option(ChannelOption.ALLOCATOR, pooledAllocator); if (conf.receiveBuf() > 0) { @@ -280,9 +280,10 @@ public void initChannel(SocketChannel ch) { // Connect to the remote server long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); - if (!cf.await(conf.connectionTimeoutMs())) { + if (!cf.await(conf.connectionCreationTimeoutMs())) { throw new IOException( - String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + String.format("Connecting to %s timed out (%s ms)", + address, conf.connectionCreationTimeoutMs())); } else if (cf.cause() != null) { throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index d305dfa8e8..f051042a7a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -19,6 +19,7 @@ import java.util.Locale; import java.util.Properties; +import java.util.concurrent.TimeUnit; import com.google.common.primitives.Ints; import io.netty.util.NettyRuntime; @@ -31,6 +32,7 @@ public class TransportConf { private final String SPARK_NETWORK_IO_MODE_KEY; private final String SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY; private final String SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY; + private final String SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY; private final String SPARK_NETWORK_IO_BACKLOG_KEY; private final String SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY; private final String SPARK_NETWORK_IO_SERVERTHREADS_KEY; @@ -54,6 +56,7 @@ public TransportConf(String module, ConfigProvider conf) { SPARK_NETWORK_IO_MODE_KEY = getConfKey("io.mode"); SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY = getConfKey("io.preferDirectBufs"); SPARK_NETWORK_IO_CONNECTIONTIMEOUT_KEY = getConfKey("io.connectionTimeout"); + SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY = getConfKey("io.connectionCreationTimeout"); SPARK_NETWORK_IO_BACKLOG_KEY = getConfKey("io.backLog"); SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY = getConfKey("io.numConnectionsPerPeer"); SPARK_NETWORK_IO_SERVERTHREADS_KEY = getConfKey("io.serverThreads"); @@ -94,7 +97,7 @@ public boolean preferDirectBufs() { return conf.getBoolean(SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); } - /** Connect timeout in milliseconds. Default 120 secs. */ + /** Connection idle timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( conf.get("spark.network.timeout", "120s")); @@ -103,6 +106,14 @@ public int connectionTimeoutMs() { return (int) defaultTimeoutMs; } + /** Connect creation timeout in milliseconds. Default 30 secs. */ + public int connectionCreationTimeoutMs() { + long connectionTimeoutS = TimeUnit.MILLISECONDS.toSeconds(connectionTimeoutMs()); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get(SPARK_NETWORK_IO_CONNECTIONCREATIONTIMEOUT_KEY, connectionTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; + } + /** Number of concurrent connections between two nodes for fetching data. */ public int numConnectionsPerPeer() { return conf.getInt(SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY, 1); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 256789b8c7..3dbee1b13d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -31,7 +31,6 @@ import scala.Tuple2; import scala.collection.Iterator; -import com.google.common.annotations.VisibleForTesting; import com.google.common.io.Closeables; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -178,8 +177,8 @@ public void write(Iterator> records) throws IOException { } } - @VisibleForTesting - long[] getPartitionLengths() { + @Override + public long[] getPartitionLengths() { return partitionLengths; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 79e38a824f..e8f94ba8ff 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -88,6 +88,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; + @Nullable private long[] partitionLengths; private long peakMemoryUsedBytes = 0; /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ @@ -219,7 +220,6 @@ void closeAndWriteOutput() throws IOException { serOutputStream = null; final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; - final long[] partitionLengths; try { partitionLengths = mergeSpills(spills); } finally { @@ -543,4 +543,9 @@ public void close() throws IOException { channel.close(); } } + + @Override + public long[] getPartitionLengths() { + return partitionLengths; + } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index dda8ed4c23..c38327cae8 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -104,11 +104,14 @@ public static UnsafeExternalSorter createWithExistingInMemorySorter( int initialSize, long pageSizeBytes, int numElementsForSpillThreshold, - UnsafeInMemorySorter inMemorySorter) throws IOException { + UnsafeInMemorySorter inMemorySorter, + long existingMemoryConsumption) throws IOException { UnsafeExternalSorter sorter = new UnsafeExternalSorter(taskMemoryManager, blockManager, serializerManager, taskContext, recordComparatorSupplier, prefixComparator, initialSize, pageSizeBytes, numElementsForSpillThreshold, inMemorySorter, false /* ignored */); sorter.spill(Long.MAX_VALUE, sorter); + taskContext.taskMetrics().incMemoryBytesSpilled(existingMemoryConsumption); + sorter.totalSpillBytes += existingMemoryConsumption; // The external sorter will be used to insert records, in-memory sorter is not needed. sorter.inMemSorter = null; return sorter; @@ -496,6 +499,7 @@ public void insertKVRecord(Object keyBase, long keyOffset, int keyLen, */ public void merge(UnsafeExternalSorter other) throws IOException { other.spill(); + totalSpillBytes += other.totalSpillBytes; spillWriters.addAll(other.spillWriters); // remove them from `spillWriters`, or the files will be deleted in `cleanupResources`. other.spillWriters.clear(); diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 61ab635842..a83762ff01 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -798,7 +798,11 @@ private[spark] class ExecutorAllocationManager( } if (taskEnd.taskInfo.speculative) { stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove{taskIndex}} - stageAttemptToNumSpeculativeTasks(stageAttempt) -= 1 + // If the previous task attempt succeeded first and it was the last task in a stage, + // the stage may have been removed before handing this speculative TaskEnd event. + if (stageAttemptToNumSpeculativeTasks.contains(stageAttempt)) { + stageAttemptToNumSpeculativeTasks(stageAttempt) -= 1 + } } taskEnd.reason match { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index f49cb3c2b8..8d9f2be621 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -244,7 +244,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( /* backlog */ 1, InetAddress.getByName("localhost"))) // A call to accept() for ServerSocket shall block infinitely. - serverSocket.map(_.setSoTimeout(0)) + serverSocket.foreach(_.setSoTimeout(0)) new Thread("accept-connections") { setDaemon(true) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c58009c166..3865c9c987 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -47,7 +47,7 @@ import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ -import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.{FetchFailedException, ShuffleBlockPusher} import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} import org.apache.spark.util._ import org.apache.spark.util.io.ChunkedByteBuffer @@ -325,6 +325,7 @@ private[spark] class Executor( case NonFatal(e) => logWarning("Unable to stop heartbeater", e) } + ShuffleBlockPusher.stop() threadPool.shutdown() // Notify plugins that executor is shutting down so they can terminate cleanly diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index adaf92d5a8..84c6647028 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -2030,4 +2030,33 @@ package object config { .version("3.1.0") .doubleConf .createWithDefault(5) + + private[spark] val SHUFFLE_NUM_PUSH_THREADS = + ConfigBuilder("spark.shuffle.push.numPushThreads") + .doc("Specify the number of threads in the block pusher pool. These threads assist " + + "in creating connections and pushing blocks to remote shuffle services. By default, the " + + "threadpool size is equal to the number of spark executor cores.") + .version("3.2.0") + .intConf + .createOptional + + private[spark] val SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH = + ConfigBuilder("spark.shuffle.push.maxBlockSizeToPush") + .doc("The max size of an individual block to push to the remote shuffle services. Blocks " + + "larger than this threshold are not pushed to be merged remotely. These shuffle blocks " + + "will be fetched by the executors in the original manner.") + .version("3.2.0") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("1m") + + private[spark] val SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH = + ConfigBuilder("spark.shuffle.push.maxBlockBatchSize") + .doc("The max size of a batch of shuffle blocks to be grouped into a single push request.") + .version("3.2.0") + .bytesConf(ByteUnit.BYTE) + // Default is 3m because it is greater than 2m which is the default value for + // TransportConf#memoryMapBytes. If this defaults to 2m as well it is very likely that each + // batch of block will be loaded in memory with memory mapping, which has higher overhead + // with small MB sized chunk of data. + .createWithDefaultString("3m") } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala new file mode 100644 index 0000000000..88d084ce1b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala @@ -0,0 +1,450 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.shuffle + +import java.io.File +import java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.ExecutorService + +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} + +import com.google.common.base.Throwables + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkEnv} +import org.apache.spark.annotation.Since +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf +import org.apache.spark.shuffle.ShuffleBlockPusher._ +import org.apache.spark.storage.{BlockId, BlockManagerId, ShufflePushBlockId} +import org.apache.spark.util.{ThreadUtils, Utils} + +/** + * Used for pushing shuffle blocks to remote shuffle services when push shuffle is enabled. + * When push shuffle is enabled, it is created after the shuffle writer finishes writing the shuffle + * file and initiates the block push process. + * + * @param conf spark configuration + */ +@Since("3.2.0") +private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging { + private[this] val maxBlockSizeToPush = conf.get(SHUFFLE_MAX_BLOCK_SIZE_TO_PUSH) + private[this] val maxBlockBatchSize = conf.get(SHUFFLE_MAX_BLOCK_BATCH_SIZE_FOR_PUSH) + private[this] val maxBytesInFlight = + conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024 + private[this] val maxReqsInFlight = conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue) + private[this] val maxBlocksInFlightPerAddress = conf.get(REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS) + private[this] var bytesInFlight = 0L + private[this] var reqsInFlight = 0 + private[this] val numBlocksInFlightPerAddress = new HashMap[BlockManagerId, Int]() + private[this] val deferredPushRequests = new HashMap[BlockManagerId, Queue[PushRequest]]() + private[this] val pushRequests = new Queue[PushRequest] + private[this] val errorHandler = createErrorHandler() + // VisibleForTesting + private[shuffle] val unreachableBlockMgrs = new HashSet[BlockManagerId]() + + // VisibleForTesting + private[shuffle] def createErrorHandler(): BlockPushErrorHandler = { + new BlockPushErrorHandler() { + // For a connection exception against a particular host, we will stop pushing any + // blocks to just that host and continue push blocks to other hosts. So, here push of + // all blocks will only stop when it is "Too Late". Also see updateStateAndCheckIfPushMore. + override def shouldRetryError(t: Throwable): Boolean = { + // If the block is too late, there is no need to retry it + !Throwables.getStackTraceAsString(t).contains(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX) + } + } + } + + /** + * Initiates the block push. + * + * @param dataFile mapper generated shuffle data file + * @param partitionLengths array of shuffle block size so we can tell shuffle block + * @param dep shuffle dependency to get shuffle ID and the location of remote shuffle + * services to push local shuffle blocks + * @param mapIndex map index of the shuffle map task + */ + private[shuffle] def initiateBlockPush( + dataFile: File, + partitionLengths: Array[Long], + dep: ShuffleDependency[_, _, _], + mapIndex: Int): Unit = { + val numPartitions = dep.partitioner.numPartitions + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") + val requests = prepareBlockPushRequests(numPartitions, mapIndex, dep.shuffleId, dataFile, + partitionLengths, dep.getMergerLocs, transportConf) + // Randomize the orders of the PushRequest, so different mappers pushing blocks at the same + // time won't be pushing the same ranges of shuffle partitions. + pushRequests ++= Utils.randomize(requests) + + submitTask(() => { + pushUpToMax() + }) + } + + /** + * Triggers the push. It's a separate method for testing. + * VisibleForTesting + */ + protected def submitTask(task: Runnable): Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.execute(task) + } + } + + /** + * Since multiple block push threads could potentially be calling pushUpToMax for the same + * mapper, we synchronize access to this method so that only one thread can push blocks for + * a given mapper. This helps to simplify access to the shared states. The down side of this + * is that we could unnecessarily block other mappers' block pushes if all the threads + * are occupied by block pushes from the same mapper. + * + * This code is similar to ShuffleBlockFetcherIterator#fetchUpToMaxBytes in how it throttles + * the data transfer between shuffle client/server. + */ + private def pushUpToMax(): Unit = synchronized { + // Process any outstanding deferred push requests if possible. + if (deferredPushRequests.nonEmpty) { + for ((remoteAddress, defReqQueue) <- deferredPushRequests) { + while (isRemoteBlockPushable(defReqQueue) && + !isRemoteAddressMaxedOut(remoteAddress, defReqQueue.front)) { + val request = defReqQueue.dequeue() + logDebug(s"Processing deferred push request for $remoteAddress with " + + s"${request.blocks.length} blocks") + sendRequest(request) + if (defReqQueue.isEmpty) { + deferredPushRequests -= remoteAddress + } + } + } + } + + // Process any regular push requests if possible. + while (isRemoteBlockPushable(pushRequests)) { + val request = pushRequests.dequeue() + val remoteAddress = request.address + if (isRemoteAddressMaxedOut(remoteAddress, request)) { + logDebug(s"Deferring push request for $remoteAddress with ${request.blocks.size} blocks") + deferredPushRequests.getOrElseUpdate(remoteAddress, new Queue[PushRequest]()) + .enqueue(request) + } else { + sendRequest(request) + } + } + + def isRemoteBlockPushable(pushReqQueue: Queue[PushRequest]): Boolean = { + pushReqQueue.nonEmpty && + (bytesInFlight == 0 || + (reqsInFlight + 1 <= maxReqsInFlight && + bytesInFlight + pushReqQueue.front.size <= maxBytesInFlight)) + } + + // Checks if sending a new push request will exceed the max no. of blocks being pushed to a + // given remote address. + def isRemoteAddressMaxedOut(remoteAddress: BlockManagerId, request: PushRequest): Boolean = { + (numBlocksInFlightPerAddress.getOrElse(remoteAddress, 0) + + request.blocks.size) > maxBlocksInFlightPerAddress + } + } + + /** + * Push blocks to remote shuffle server. The callback listener will invoke #pushUpToMax again + * to trigger pushing the next batch of blocks once some block transfer is done in the current + * batch. This way, we decouple the map task from the block push process, since it is netty + * client thread instead of task execution thread which takes care of majority of the block + * pushes. + */ + private def sendRequest(request: PushRequest): Unit = { + bytesInFlight += request.size + reqsInFlight += 1 + numBlocksInFlightPerAddress(request.address) = numBlocksInFlightPerAddress.getOrElseUpdate( + request.address, 0) + request.blocks.length + + val sizeMap = request.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val address = request.address + val blockIds = request.blocks.map(_._1.toString) + val remainingBlocks = new HashSet[String]() ++= blockIds + + val blockPushListener = new BlockFetchingListener { + // Initiating a connection and pushing blocks to a remote shuffle service is always handled by + // the block-push-threads. We should not initiate the connection creation in the + // blockPushListener callbacks which are invoked by the netty eventloop because: + // 1. TrasportClient.createConnection(...) blocks for connection to be established and it's + // recommended to avoid any blocking operations in the eventloop; + // 2. The actual connection creation is a task that gets added to the task queue of another + // eventloop which could have eventloops eventually blocking each other. + // Once the blockPushListener is notified of the block push success or failure, we + // just delegate it to block-push-threads. + def handleResult(result: PushResult): Unit = { + submitTask(() => { + if (updateStateAndCheckIfPushMore( + sizeMap(result.blockId), address, remainingBlocks, result)) { + pushUpToMax() + } + }) + } + + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + logTrace(s"Push for block $blockId to $address successful.") + handleResult(PushResult(blockId, null)) + } + + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + // check the message or it's cause to see it needs to be logged. + if (!errorHandler.shouldLogError(exception)) { + logTrace(s"Pushing block $blockId to $address failed.", exception) + } else { + logWarning(s"Pushing block $blockId to $address failed.", exception) + } + handleResult(PushResult(blockId, exception)) + } + } + SparkEnv.get.blockManager.blockStoreClient.pushBlocks( + address.host, address.port, blockIds.toArray, + sliceReqBufferIntoBlockBuffers(request.reqBuffer, request.blocks.map(_._2)), + blockPushListener) + } + + /** + * Given the ManagedBuffer representing all the continuous blocks inside the shuffle data file + * for a PushRequest and an array of individual block sizes, load the buffer from disk into + * memory and slice it into multiple smaller buffers representing each block. + * + * With nio ByteBuffer, the individual block buffers share data with the initial in memory + * buffer loaded from disk. Thus only one copy of the block data is kept in memory. + * @param reqBuffer A {{FileSegmentManagedBuffer}} representing all the continuous blocks in + * the shuffle data file for a PushRequest + * @param blockSizes Array of block sizes + * @return Array of in memory buffer for each individual block + */ + private def sliceReqBufferIntoBlockBuffers( + reqBuffer: ManagedBuffer, + blockSizes: Seq[Int]): Array[ManagedBuffer] = { + if (blockSizes.size == 1) { + Array(reqBuffer) + } else { + val inMemoryBuffer = reqBuffer.nioByteBuffer() + val blockOffsets = new Array[Int](blockSizes.size) + var offset = 0 + for (index <- blockSizes.indices) { + blockOffsets(index) = offset + offset += blockSizes(index) + } + blockOffsets.zip(blockSizes).map { + case (offset, size) => + new NioManagedBuffer(inMemoryBuffer.duplicate() + .position(offset) + .limit(offset + size).asInstanceOf[ByteBuffer].slice()) + }.toArray + } + } + + /** + * Updates the stats and based on the previous push result decides whether to push more blocks + * or stop. + * + * @param bytesPushed number of bytes pushed. + * @param address address of the remote service + * @param remainingBlocks remaining blocks + * @param pushResult result of the last push + * @return true if more blocks should be pushed; false otherwise. + */ + private def updateStateAndCheckIfPushMore( + bytesPushed: Long, + address: BlockManagerId, + remainingBlocks: HashSet[String], + pushResult: PushResult): Boolean = synchronized { + remainingBlocks -= pushResult.blockId + bytesInFlight -= bytesPushed + numBlocksInFlightPerAddress(address) = numBlocksInFlightPerAddress(address) - 1 + if (remainingBlocks.isEmpty) { + reqsInFlight -= 1 + } + if (pushResult.failure != null && pushResult.failure.getCause.isInstanceOf[ConnectException]) { + // Remove all the blocks for this address just once because removing from pushRequests + // is expensive. If there is a ConnectException for the first block, all the subsequent + // blocks to that address will fail, so should avoid removing multiple times. + if (!unreachableBlockMgrs.contains(address)) { + var removed = 0 + unreachableBlockMgrs.add(address) + removed += pushRequests.dequeueAll(req => req.address == address).length + removed += deferredPushRequests.remove(address).map(_.length).getOrElse(0) + logWarning(s"Received a ConnectException from $address. " + + s"Dropping $removed push-requests and " + + s"not pushing any more blocks to this address.") + } + } + if (pushResult.failure != null && !errorHandler.shouldRetryError(pushResult.failure)) { + logDebug(s"Received after merge is finalized from $address. Not pushing any more blocks.") + return false + } else { + remainingBlocks.isEmpty && (pushRequests.nonEmpty || deferredPushRequests.nonEmpty) + } + } + + /** + * Convert the shuffle data file of the current mapper into a list of PushRequest. Basically, + * continuous blocks in the shuffle file are grouped into a single request to allow more + * efficient read of the block data. Each mapper for a given shuffle will receive the same + * list of BlockManagerIds as the target location to push the blocks to. All mappers in the + * same shuffle will map shuffle partition ranges to individual target locations in a consistent + * manner to make sure each target location receives shuffle blocks belonging to the same set + * of partition ranges. 0-length blocks and blocks that are large enough will be skipped. + * + * @param numPartitions sumber of shuffle partitions in the shuffle file + * @param partitionId map index of the current mapper + * @param shuffleId shuffleId of current shuffle + * @param dataFile shuffle data file + * @param partitionLengths array of sizes of blocks in the shuffle data file + * @param mergerLocs target locations to push blocks to + * @param transportConf transportConf used to create FileSegmentManagedBuffer + * @return List of the PushRequest, randomly shuffled. + * + * VisibleForTesting + */ + private[shuffle] def prepareBlockPushRequests( + numPartitions: Int, + partitionId: Int, + shuffleId: Int, + dataFile: File, + partitionLengths: Array[Long], + mergerLocs: Seq[BlockManagerId], + transportConf: TransportConf): Seq[PushRequest] = { + var offset = 0L + var currentReqSize = 0 + var currentReqOffset = 0L + var currentMergerId = 0 + val numMergers = mergerLocs.length + val requests = new ArrayBuffer[PushRequest] + var blocks = new ArrayBuffer[(BlockId, Int)] + for (reduceId <- 0 until numPartitions) { + val blockSize = partitionLengths(reduceId) + logDebug( + s"Block ${ShufflePushBlockId(shuffleId, partitionId, reduceId)} is of size $blockSize") + // Skip 0-length blocks and blocks that are large enough + if (blockSize > 0) { + val mergerId = math.min(math.floor(reduceId * 1.0 / numPartitions * numMergers), + numMergers - 1).asInstanceOf[Int] + // Start a new PushRequest if the current request goes beyond the max batch size, + // or the number of blocks in the current request goes beyond the limit per destination, + // or the next block push location is for a different shuffle service, or the next block + // exceeds the max block size to push limit. This guarantees that each PushRequest + // represents continuous blocks in the shuffle file to be pushed to the same shuffle + // service, and does not go beyond existing limitations. + if (currentReqSize + blockSize <= maxBlockBatchSize + && blocks.size < maxBlocksInFlightPerAddress + && mergerId == currentMergerId && blockSize <= maxBlockSizeToPush) { + // Add current block to current batch + currentReqSize += blockSize.toInt + } else { + if (blocks.nonEmpty) { + // Convert the previous batch into a PushRequest + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + blocks = new ArrayBuffer[(BlockId, Int)] + } + // Start a new batch + currentReqSize = 0 + // Set currentReqOffset to -1 so we are able to distinguish between the initial value + // of currentReqOffset and when we are about to start a new batch + currentReqOffset = -1 + currentMergerId = mergerId + } + // Only push blocks under the size limit + if (blockSize <= maxBlockSizeToPush) { + val blockSizeInt = blockSize.toInt + blocks += ((ShufflePushBlockId(shuffleId, partitionId, reduceId), blockSizeInt)) + // Only update currentReqOffset if the current block is the first in the request + if (currentReqOffset == -1) { + currentReqOffset = offset + } + if (currentReqSize == 0) { + currentReqSize += blockSizeInt + } + } + } + offset += blockSize + } + // Add in the final request + if (blocks.nonEmpty) { + requests += PushRequest(mergerLocs(currentMergerId), blocks.toSeq, + createRequestBuffer(transportConf, dataFile, currentReqOffset, currentReqSize)) + } + requests.toSeq + } + + // Visible for testing + protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + new FileSegmentManagedBuffer(conf, dataFile, offset, length) + } +} + +private[spark] object ShuffleBlockPusher { + + /** + * A request to push blocks to a remote shuffle service + * @param address remote shuffle service location to push blocks to + * @param blocks list of block IDs and their sizes + * @param reqBuffer a chunk of data in the shuffle data file corresponding to the continuous + * blocks represented in this request + */ + private[spark] case class PushRequest( + address: BlockManagerId, + blocks: Seq[(BlockId, Int)], + reqBuffer: ManagedBuffer) { + val size = blocks.map(_._2).sum + } + + /** + * Result of the block push. + * @param blockId blockId + * @param failure exception if the push was unsuccessful; null otherwise; + */ + private case class PushResult(blockId: String, failure: Throwable) + + private val BLOCK_PUSHER_POOL: ExecutorService = { + val conf = SparkEnv.get.conf + if (Utils.isPushBasedShuffleEnabled(conf)) { + val numThreads = conf.get(SHUFFLE_NUM_PUSH_THREADS) + .getOrElse(conf.getInt(SparkLauncher.EXECUTOR_CORES, 1)) + ThreadUtils.newDaemonFixedThreadPool(numThreads, "shuffle-block-push-thread") + } else { + null + } + } + + /** + * Stop the shuffle pusher pool if it isn't null. + */ + private[spark] def stop(): Unit = { + if (BLOCK_PUSHER_POOL != null) { + BLOCK_PUSHER_POOL.shutdown() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala index 1429144c6f..abff650b06 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala @@ -21,6 +21,7 @@ import org.apache.spark.{Partition, ShuffleDependency, SparkEnv, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.MapStatus +import org.apache.spark.util.Utils /** * The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor @@ -57,7 +58,23 @@ private[spark] class ShuffleWriteProcessor extends Serializable with Logging { createMetricsReporter(context)) writer.write( rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) - writer.stop(success = true).get + val mapStatus = writer.stop(success = true) + if (mapStatus.isDefined) { + // Initiate shuffle push process if push based shuffle is enabled + // The map task only takes care of converting the shuffle data file into multiple + // block push requests. It delegates pushing the blocks to a different thread-pool - + // ShuffleBlockPusher.BLOCK_PUSHER_POOL. + if (Utils.isPushBasedShuffleEnabled(SparkEnv.get.conf) && dep.getMergerLocs.nonEmpty) { + manager.shuffleBlockResolver match { + case resolver: IndexShuffleBlockResolver => + val dataFile = resolver.getDataFile(dep.shuffleId, mapId) + new ShuffleBlockPusher(SparkEnv.get.conf) + .initiateBlockPush(dataFile, writer.getPartitionLengths(), dep, partition.index) + case _ => + } + } + } + mapStatus.get } catch { case e: Exception => try { diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala index 4cc4ef5f18..a279b4c8f4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriter.scala @@ -31,4 +31,7 @@ private[spark] abstract class ShuffleWriter[K, V] { /** Close this writer, passing along whether the map completed */ def stop(success: Boolean): Option[MapStatus] + + /** Get the lengths of each partition */ + def getPartitionLengths(): Array[Long] } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 83ebe3e129..af8d1e2fff 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -45,6 +45,8 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null + private var partitionLengths: Array[Long] = _ + private val writeMetrics = context.taskMetrics().shuffleWriteMetrics /** Write a bunch of records to this task's output */ @@ -67,7 +69,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = shuffleExecutorComponents.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - val partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths + partitionLengths = mapOutputWriter.commitAllPartitions().getPartitionLengths mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, mapId) } @@ -93,6 +95,8 @@ private[spark] class SortShuffleWriter[K, V, C]( } } } + + override def getPartitionLengths(): Array[Long] = partitionLengths } private[spark] object SortShuffleWriter { diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index bf19897e51..52d41cdd72 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -366,10 +366,12 @@ private[spark] class AppStatusListener( // Implicitly exclude every available executor for the stage associated with this node Option(liveStages.get((stageId, stageAttemptId))).foreach { stage => - val executorIds = liveExecutors.values.filter(_.host == hostId).map(_.executorId).toSeq + val executorIds = liveExecutors.values.filter(exec => exec.host == hostId + && exec.executorId != SparkContext.DRIVER_IDENTIFIER).map(_.executorId).toSeq setStageExcludedStatus(stage, now, executorIds: _*) } - liveExecutors.values.filter(_.hostname == hostId).foreach { exec => + liveExecutors.values.filter(exec => exec.hostname == hostId + && exec.executorId != SparkContext.DRIVER_IDENTIFIER).foreach { exec => addExcludedStageTo(exec, stageId, now) } } @@ -416,7 +418,7 @@ private[spark] class AppStatusListener( // Implicitly (un)exclude every executor associated with the node. liveExecutors.values.foreach { exec => - if (exec.hostname == host) { + if (exec.hostname == host && exec.executorId != SparkContext.DRIVER_IDENTIFIER) { updateExecExclusionStatus(exec, excluded, now) } } @@ -757,6 +759,7 @@ private[spark] class AppStatusListener( exec.completedTasks += completedDelta exec.failedTasks += failedDelta exec.totalDuration += event.taskInfo.duration + exec.peakExecutorMetrics.compareAndUpdatePeakValues(event.taskExecutorMetrics) // Note: For resubmitted tasks, we continue to use the metrics that belong to the // first attempt of this task. This may not be 100% accurate because the first attempt diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 7b084e73c9..73bf809a08 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -20,7 +20,7 @@ package org.apache.spark.storage import java.util.UUID import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Since} /** * :: DeveloperApi :: @@ -81,6 +81,12 @@ case class ShuffleIndexBlockId(shuffleId: Int, mapId: Long, reduceId: Int) exten override def name: String = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" } +@Since("3.2.0") +@DeveloperApi +case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) extends BlockId { + override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId +} + @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { override def name: String = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) @@ -122,6 +128,7 @@ object BlockId { val SHUFFLE_BATCH = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r + val SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -140,6 +147,8 @@ object BlockId { ShuffleDataBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt) + case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) => + ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index c18a2e31df..be12507587 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -21,6 +21,28 @@ "addTime" : "2015-02-03T16:43:00.906GMT", "executorLogs" : { }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index 9adda275b5..0a3eb81140 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -16,7 +16,7 @@ "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, - "isBlacklisted" : true, + "isBlacklisted" : false, "maxMemory" : 908381388, "addTime" : "2016-11-16T22:33:31.477GMT", "executorLogs" : { }, @@ -30,7 +30,7 @@ "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, - "isExcluded" : true, + "isExcluded" : false, "excludedInStages" : [ ] }, { "id" : "3", @@ -64,6 +64,28 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -101,6 +123,28 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -138,6 +182,28 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -175,6 +241,28 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json index 65bd309c10..8869fb4e29 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_expectation.json @@ -16,7 +16,7 @@ "totalInputBytes" : 0, "totalShuffleRead" : 0, "totalShuffleWrite" : 0, - "isBlacklisted" : true, + "isBlacklisted" : false, "maxMemory" : 908381388, "addTime" : "2016-11-16T22:33:31.477GMT", "executorLogs" : { }, @@ -30,7 +30,7 @@ "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, - "isExcluded" : true, + "isExcluded" : false, "excludedInStages" : [ ] }, { "id" : "3", @@ -64,6 +64,28 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -101,6 +123,28 @@ "totalOffHeapStorageMemory" : 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -138,6 +182,28 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -175,6 +241,28 @@ "totalOffHeapStorageMemory": 524288000 }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json index 46e8f81d0e..21cc9d0812 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_excludeOnFailure_unexcluding_expectation.json @@ -52,6 +52,28 @@ "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -83,6 +105,28 @@ "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -114,6 +158,28 @@ "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, @@ -145,6 +211,28 @@ "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" }, "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 0, + "JVMOffHeapMemory" : 0, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 0, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 0, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 0, + "ProcessTreeJVMRSSMemory" : 0, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 0, + "MinorGCTime" : 0, + "MajorGCCount" : 0, + "MajorGCTime" : 0 + }, "attributes" : { }, "resources" : { }, "resourceProfileId" : 0, diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 770ffeef41..f36789c5a1 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1067,17 +1067,17 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu dependencyJars.foreach(jar => assert(sc.listJars().exists(_.contains(jar)))) assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( - "Added dependency jars of Ivy URI" + - " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) + "Added dependency jars of Ivy URI " + + "ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) // test dependency jars exist sc.addJar("ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true") assert(logAppender.loggingEvents.count(_.getRenderedMessage.contains( - "The dependency jars of Ivy URI" + - " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) + "The dependency jars of Ivy URI " + + "ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) == 1) val existMsg = logAppender.loggingEvents.filter(_.getRenderedMessage.contains( - "The dependency jars of Ivy URI" + - " ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) + "The dependency jars of Ivy URI " + + "ivy://org.apache.hive:hive-storage-api:2.7.0?transitive=true")) .head.getRenderedMessage dependencyJars.foreach(jar => assert(existMsg.contains(jar))) } @@ -1109,8 +1109,8 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu "invalidParam1=foo&invalidParam2=boo") assert(sc.listJars().exists(_.contains("org.apache.hive_hive-storage-api-2.7.0.jar"))) assert(logAppender.loggingEvents.exists(_.getRenderedMessage.contains( - "Invalid parameters `invalidParam1,invalidParam2` found in Ivy URI query" + - " `invalidParam1=foo&invalidParam2=boo`."))) + "Invalid parameters `invalidParam1,invalidParam2` found in Ivy URI query " + + "`invalidParam1=foo&invalidParam2=boo`."))) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala new file mode 100644 index 0000000000..cc561e6106 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleBlockPusherSuite.scala @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.shuffle + +import java.io.File +import java.net.ConnectException +import java.nio.ByteBuffer +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer + +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark._ +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, BlockStoreClient} +import org.apache.spark.network.shuffle.ErrorHandler.BlockPushErrorHandler +import org.apache.spark.network.util.TransportConf +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.ShuffleBlockPusher.PushRequest +import org.apache.spark.storage._ + +class ShuffleBlockPusherSuite extends SparkFunSuite with BeforeAndAfterEach { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockManager: BlockManager = _ + @Mock(answer = RETURNS_SMART_NULLS) private var dependency: ShuffleDependency[Int, Int, Int] = _ + @Mock(answer = RETURNS_SMART_NULLS) private var shuffleClient: BlockStoreClient = _ + + private var conf: SparkConf = _ + private var pushedBlocks = new ArrayBuffer[String] + + override def beforeEach(): Unit = { + super.beforeEach() + conf = new SparkConf(loadDefaults = false) + MockitoAnnotations.initMocks(this) + when(dependency.partitioner).thenReturn(new HashPartitioner(8)) + when(dependency.serializer).thenReturn(new JavaSerializer(conf)) + when(dependency.getMergerLocs).thenReturn(Seq(BlockManagerId("test-client", "test-client", 1))) + conf.set("spark.shuffle.push.based.enabled", "true") + conf.set("spark.shuffle.service.enabled", "true") + // Set the env because the shuffler writer gets the shuffle client instance from the env. + val mockEnv = mock(classOf[SparkEnv]) + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.blockManager).thenReturn(blockManager) + SparkEnv.set(mockEnv) + when(blockManager.blockStoreClient).thenReturn(shuffleClient) + } + + override def afterEach(): Unit = { + pushedBlocks.clear() + super.afterEach() + } + + private def interceptPushedBlocksForSuccess(): Unit = { + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + }) + } + + private def verifyPushRequests( + pushRequests: Seq[PushRequest], + expectedSizes: Seq[Int]): Unit = { + (pushRequests, expectedSizes).zipped.foreach((req, size) => { + assert(req.size == size) + }) + } + + test("A batch of blocks is limited by maxBlocksBatchSize") { + conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") + conf.set("spark.shuffle.push.maxBlockSizeToPush", "2048k") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val largeBlockSize = 2 * 1024 * 1024 + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, largeBlockSize, largeBlockSize), mergerLocs, + mock(classOf[TransportConf])) + assert(pushRequests.length == 3) + verifyPushRequests(pushRequests, Seq(6, largeBlockSize, largeBlockSize)) + } + + test("Large blocks are excluded in the preparation") { + conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, 1028, 1024), mergerLocs, mock(classOf[TransportConf])) + assert(pushRequests.length == 2) + verifyPushRequests(pushRequests, Seq(6, 1024)) + } + + test("Number of blocks in a push request are limited by maxBlocksInFlightPerAddress ") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val blockPusher = new TestShuffleBlockPusher(conf) + val mergerLocs = dependency.getMergerLocs.map(loc => BlockManagerId("", loc.host, loc.port)) + val pushRequests = blockPusher.prepareBlockPushRequests(5, 0, 0, + mock(classOf[File]), Array(2, 2, 2, 2, 2), mergerLocs, mock(classOf[TransportConf])) + assert(pushRequests.length == 5) + verifyPushRequests(pushRequests, Seq(2, 2, 2, 2, 2)) + } + + test("Basic block push") { + interceptPushedBlocksForSuccess() + val blockPusher = new TestShuffleBlockPusher(conf) + blockPusher.initiateBlockPush(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + blockPusher.runPendingTasks() + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + ShuffleBlockPusher.stop() + } + + test("Large blocks are skipped for push") { + conf.set("spark.shuffle.push.maxBlockSizeToPush", "1k") + interceptPushedBlocksForSuccess() + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( + mock(classOf[File]), Array(2, 2, 2, 2, 2, 2, 2, 1100), dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions - 1) + ShuffleBlockPusher.stop() + } + + test("Number of blocks in flight per address are limited by maxBlocksInFlightPerAddress") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + interceptPushedBlocksForSuccess() + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(8)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + ShuffleBlockPusher.stop() + } + + test("Hit maxBlocksInFlightPerAddress limit so that the blocks are deferred") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + var blockPendingResponse : String = null + var listener : BlockFetchingListener = null + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks + val managedBuffers = invocation.getArguments()(3).asInstanceOf[Array[ManagedBuffer]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + // Expecting 2 blocks + assert(blocks.length == 2) + if (blockPendingResponse == null) { + blockPendingResponse = blocks(1) + listener = blockFetchListener + // Respond with success only for the first block which will cause all the rest of the + // blocks to be deferred + blockFetchListener.onBlockFetchSuccess(blocks(0), managedBuffers(0)) + } else { + (blocks, managedBuffers).zipped.foreach((blockId, buffer) => { + blockFetchListener.onBlockFetchSuccess(blockId, buffer) + }) + } + }) + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 2) + // this will trigger push of deferred blocks + listener.onBlockFetchSuccess(blockPendingResponse, mock(classOf[ManagedBuffer])) + pusher.runPendingTasks() + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 8) + ShuffleBlockPusher.stop() + } + + test("Number of shuffle blocks grouped in a single push request is limited by " + + "maxBlockBatchSize") { + conf.set("spark.shuffle.push.maxBlockBatchSize", "1m") + interceptPushedBlocksForSuccess() + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush(mock(classOf[File]), + Array.fill(dependency.partitioner.numPartitions) { 512 * 1024 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(4)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == dependency.partitioner.numPartitions) + ShuffleBlockPusher.stop() + } + + test("Error retries") { + val pusher = new ShuffleBlockPusher(conf) + val errorHandler = pusher.createErrorHandler() + assert( + !errorHandler.shouldRetryError(new RuntimeException( + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) + assert(errorHandler.shouldRetryError(new RuntimeException(new ConnectException()))) + assert( + errorHandler.shouldRetryError(new RuntimeException(new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) + assert (errorHandler.shouldRetryError(new Throwable())) + } + + test("Error logging") { + val pusher = new ShuffleBlockPusher(conf) + val errorHandler = pusher.createErrorHandler() + assert( + !errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)))) + assert(!errorHandler.shouldLogError(new RuntimeException( + new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX)))) + assert(errorHandler.shouldLogError(new Throwable())) + } + + test("Blocks are continued to push even when a block push fails with collision " + + "exception") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val pusher = new TestShuffleBlockPusher(conf) + var failBlock: Boolean = true + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + blocks.foreach(blockId => { + if (failBlock) { + failBlock = false + // Fail the first block with the collision exception. + blockFetchListener.onBlockFetchFailure(blockId, new RuntimeException( + new IllegalArgumentException( + BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX))) + } else { + pushedBlocks += blockId + blockFetchListener.onBlockFetchSuccess(blockId, mock(classOf[ManagedBuffer])) + } + }) + }) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(8)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.length == 7) + } + + test("More blocks are not pushed when a block push fails with too late " + + "exception") { + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "1") + val pusher = new TestShuffleBlockPusher(conf) + var failBlock: Boolean = true + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + blocks.foreach(blockId => { + if (failBlock) { + failBlock = false + // Fail the first block with the too late exception. + blockFetchListener.onBlockFetchFailure(blockId, new RuntimeException( + new IllegalArgumentException(BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX))) + } else { + pushedBlocks += blockId + blockFetchListener.onBlockFetchSuccess(blockId, mock(classOf[ManagedBuffer])) + } + }) + }) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(1)) + .pushBlocks(any(), any(), any(), any(), any()) + assert(pushedBlocks.isEmpty) + } + + test("Connect exceptions remove all the push requests for that host") { + when(dependency.getMergerLocs).thenReturn( + Seq(BlockManagerId("client1", "client1", 1), BlockManagerId("client2", "client2", 2))) + conf.set("spark.reducer.maxBlocksInFlightPerAddress", "2") + when(shuffleClient.pushBlocks(any(), any(), any(), any(), any())) + .thenAnswer((invocation: InvocationOnMock) => { + val blocks = invocation.getArguments()(2).asInstanceOf[Array[String]] + pushedBlocks ++= blocks + val blockFetchListener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + blocks.foreach(blockId => { + blockFetchListener.onBlockFetchFailure( + blockId, new RuntimeException(new ConnectException())) + }) + }) + val pusher = new TestShuffleBlockPusher(conf) + pusher.initiateBlockPush( + mock(classOf[File]), Array.fill(dependency.partitioner.numPartitions) { 2 }, dependency, 0) + pusher.runPendingTasks() + verify(shuffleClient, times(2)) + .pushBlocks(any(), any(), any(), any(), any()) + // 2 blocks for each merger locations + assert(pushedBlocks.length == 4) + assert(pusher.unreachableBlockMgrs.size == 2) + } + + private class TestShuffleBlockPusher(conf: SparkConf) extends ShuffleBlockPusher(conf) { + private[this] val tasks = new LinkedBlockingQueue[Runnable] + + override protected def submitTask(task: Runnable): Unit = { + tasks.add(task) + } + + def runPendingTasks(): Unit = { + // This ensures that all the submitted tasks - updateStateAndCheckIfPushMore and pushUpToMax + // are run synchronously. + while (!tasks.isEmpty) { + tasks.take().run() + } + } + + override protected def createRequestBuffer( + conf: TransportConf, + dataFile: File, + offset: Long, + length: Long): ManagedBuffer = { + val managedBuffer = mock(classOf[ManagedBuffer]) + val byteBuffer = new Array[Byte](length.toInt) + when(managedBuffer.nioByteBuffer()).thenReturn(ByteBuffer.wrap(byteBuffer)) + managedBuffer + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/DependencyUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/DependencyUtilsSuite.scala new file mode 100644 index 0000000000..bf8edeff37 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/DependencyUtilsSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.util + +import java.net.URI + +import org.apache.spark.SparkFunSuite + +class DependencyUtilsSuite extends SparkFunSuite { + + test("SPARK-33084: Add jar support Ivy URI -- test invalid ivy uri") { + val e1 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies(URI.create("ivy://")) + }.getMessage + assert(e1.contains("Expected authority at index 6: ivy://")) + + val e2 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies(URI.create("ivy://org.apache.test:test-test")) + }.getMessage + assert(e2.contains("Invalid Ivy URI authority in uri ivy://org.apache.test:test-test: " + + "Expected 'org:module:version', found org.apache.test:test-test.")) + + val e3 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.test:test-test:1.0.0?foo=")) + }.getMessage + assert(e3.contains("Invalid query string in Ivy URI " + + "ivy://org.apache.test:test-test:1.0.0?foo=:")) + + val e4 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.test:test-test:1.0.0?bar=&baz=foo")) + }.getMessage + assert(e4.contains("Invalid query string in Ivy URI " + + "ivy://org.apache.test:test-test:1.0.0?bar=&baz=foo: bar=&baz=foo")) + + val e5 = intercept[IllegalArgumentException] { + DependencyUtils.resolveMavenDependencies( + URI.create("ivy://org.apache.test:test-test:1.0.0?exclude=org.apache")) + }.getMessage + assert(e5.contains("Invalid exclude string in Ivy URI " + + "ivy://org.apache.test:test-test:1.0.0?exclude=org.apache: " + + "expected 'org:module,org:module,..', found org.apache")) + } +} diff --git a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 index 9c516203dd..8d8ef2e972 100644 --- a/dev/deps/spark-deps-hadoop-2.7-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-2.7-hive-2.3 @@ -58,7 +58,7 @@ curator-recipes/2.7.1//curator-recipes-2.7.1.jar datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar -derby/10.12.1.1//derby-10.12.1.1.jar +derby/10.14.2.0//derby-10.14.2.0.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar flatbuffers-java/1.9.0//flatbuffers-java-1.9.0.jar generex/1.0.2//generex-1.0.2.jar diff --git a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 index 1d80fadb57..bf56fc18c0 100644 --- a/dev/deps/spark-deps-hadoop-3.2-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3.2-hive-2.3 @@ -55,7 +55,7 @@ curator-recipes/2.13.0//curator-recipes-2.13.0.jar datanucleus-api-jdo/4.2.4//datanucleus-api-jdo-4.2.4.jar datanucleus-core/4.1.17//datanucleus-core-4.1.17.jar datanucleus-rdbms/4.1.19//datanucleus-rdbms-4.1.19.jar -derby/10.12.1.1//derby-10.12.1.1.jar +derby/10.14.2.0//derby-10.14.2.0.jar dnsjava/2.1.7//dnsjava-2.1.7.jar dropwizard-metrics-hadoop-metrics2-reporter/0.1.2//dropwizard-metrics-hadoop-metrics2-reporter-0.1.2.jar ehcache/3.3.1//ehcache-3.3.1.jar diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index de98f29acf..f10d46763c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -84,6 +84,7 @@ MLlib (Machine Learning) GraphX (Graph Processing) SparkR (R on Spark) + PySpark (Python on Spark) diff --git a/docs/css/main.css b/docs/css/main.css index 309ad7b3bd..6710b6e856 100755 --- a/docs/css/main.css +++ b/docs/css/main.css @@ -326,7 +326,7 @@ a.anchorjs-link:hover { text-decoration: none; } border-left-width: 0px; border-bottom-width: 0px; margin-top: 0px; - width: 210px; + width: 220px; height: 80%; float: left; position: fixed; diff --git a/docs/index.md b/docs/index.md index 8fd169e63f..c4c2d722f9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -113,6 +113,8 @@ options for deployment: * [Spark Streaming](streaming-programming-guide.html): processing data streams using DStreams (old API) * [MLlib](ml-guide.html): applying machine learning algorithms * [GraphX](graphx-programming-guide.html): processing graphs +* [SparkR](sparkr.html): processing data with Spark in R +* [PySpark](api/python/getting_started/index.html): processing data with Spark in Python **API Docs:** diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md index ae3572c474..376c2042d4 100644 --- a/docs/sql-data-sources-hive-tables.md +++ b/docs/sql-data-sources-hive-tables.md @@ -139,7 +139,7 @@ The following options can be used to configure the version of Hive that is used builtin Location of the jars that should be used to instantiate the HiveMetastoreClient. This - property can be one of three options: + property can be one of four options:
  1. builtin
  2. Use Hive 2.3.7, which is bundled with the Spark assembly when -Phive is @@ -148,6 +148,9 @@ The following options can be used to configure the version of Hive that is used
  3. maven
  4. Use Hive jars of specified version downloaded from Maven repositories. This configuration is not generally recommended for production deployments. +
  5. path
  6. + Use Hive jars configured by spark.sql.hive.metastore.jars.path + in comma separated format. Support both local or remote paths.
  7. A classpath in the standard format for the JVM. This classpath must include all of Hive and its dependencies, including the correct version of Hadoop. These jars only need to be present on the driver, but if you are running in yarn cluster mode then you must ensure @@ -156,6 +159,28 @@ The following options can be used to configure the version of Hive that is used 1.4.0 + + spark.sql.hive.metastore.jars.path + (empty) + + Comma-separated paths of the jars that used to instantiate the HiveMetastoreClient. + This configuration is useful only when spark.sql.hive.metastore.jars is set as path. +
    + The paths can be any of the following format: +
      +
    1. file://path/to/jar/foo.jar
    2. +
    3. hdfs://nameservice/path/to/jar/foo.jar
    4. +
    5. /path/to/jar/(path without URI scheme follow conf fs.defaultFS's URI schema)
    6. +
    7. [http/https/ftp]://path/to/jar/foo.jar
    8. +
    + Note that 1, 2, and 3 support wildcard. For example: +
      +
    1. file://path/to/jar/*,file://path2/to/jar/*/*.jar
    2. +
    3. hdfs://nameservice/path/to/jar/*,hdfs://nameservice2/path/to/jar/*/*.jar
    4. +
    + + 3.1.0 + spark.sql.hive.metastore.sharedPrefixes com.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md index 4c4b3b1eee..f5c9677c34 100644 --- a/docs/sql-data-sources-orc.md +++ b/docs/sql-data-sources-orc.md @@ -19,12 +19,115 @@ license: | limitations under the License. --- -Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. -To do that, the following configurations are newly added. The vectorized reader is used for the -native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` -is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC -serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), -the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. +* Table of contents +{:toc} + +[Apache ORC](https://orc.apache.org) is a columnar format which has more advanced features like native zstd compression, bloom filter and columnar encryption. + +### ORC Implementation + +Spark supports two ORC implementations (`native` and `hive`) which is controlled by `spark.sql.orc.impl`. +Two implementations share most functionalities with different design goals. +- `native` implementation is designed to follow Spark's data source behavior like `Parquet`. +- `hive` implementation is designed to follow Hive's behavior and uses Hive SerDe. + +For example, historically, `native` implementation handles `CHAR/VARCHAR` with Spark's native `String` while `hive` implementation handles it via Hive `CHAR/VARCHAR`. The query results are different. Since Spark 3.1.0, [SPARK-33480](https://issues.apache.org/jira/browse/SPARK-33480) removes this difference by supporting `CHAR/VARCHAR` from Spark-side. + +### Vectorized Reader + +`native` implementation supports a vectorized ORC reader and has been the default ORC implementaion since Spark 2.3. +The vectorized reader is used for the native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. +For the Hive ORC serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), +the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`, and is turned on by default. + +### Schema Merging + +Like Protocol Buffer, Avro, and Thrift, ORC also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple ORC files with different but mutually compatible schemas. The ORC data +source is now able to automatically detect this case and merge schemas of all these files. + +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default . You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading ORC files, or +2. setting the global SQL option `spark.sql.orc.mergeSchema` to `true`. + +### Zstandard + +Spark supports both Hadoop 2 and 3. Since Spark 3.2, you can take advantage +of Zstandard compression in ORC files on both Hadoop versions. +Please see [Zstandard](https://facebook.github.io/zstd/) for the benefits. + +
    +
    + +{% highlight sql %} +CREATE TABLE compressed ( + key STRING, + value STRING +) +USING ORC +OPTIONS ( + compression 'zstd' +) +{% endhighlight %} +
    +
    + +### Bloom Filters + +You can control bloom filters and dictionary encodings for ORC data sources. The following ORC example will create bloom filter and use dictionary encoding only for `favorite_color`. To find more detailed information about the extra ORC options, visit the official Apache ORC websites. + +
    +
    + +{% highlight sql %} +CREATE TABLE users_with_options ( + name STRING, + favorite_color STRING, + favorite_numbers array +) +USING ORC +OPTIONS ( + orc.bloom.filter.columns 'favorite_color', + orc.dictionary.key.threshold '1.0', + orc.column.encoding.direct 'name' +) +{% endhighlight %} +
    +
    + +### Columnar Encryption + +Since Spark 3.2, columnar encryption is supported for ORC tables with Apache ORC 1.6. +The following example is using Hadoop KMS as a key provider with the given location. +Please visit [Apache Hadoop KMS](https://hadoop.apache.org/docs/current/hadoop-kms/index.html) for the detail. + +
    +
    +{% highlight sql %} +CREATE TABLE encrypted ( + ssn STRING, + email STRING, + name STRING +) +USING ORC +OPTIONS ( + hadoop.security.key.provider.path "kms://http@localhost:9600/kms", + orc.key.provider "hadoop", + orc.encrypt "pii:ssn,email", + orc.mask "nullify:ssn;sha256:email" +) +{% endhighlight %} +
    +
    + +### Hive metastore ORC table conversion + +When reading from Hive metastore ORC tables and inserting to Hive metastore ORC tables, Spark SQL will try to use its own ORC support instead of Hive SerDe for better performance. For CTAS statement, only non-partitioned Hive metastore ORC tables are converted. This behavior is controlled by the `spark.sql.hive.convertMetastoreOrc` configuration, and is turned on by default. + +### Configuration @@ -48,4 +151,24 @@ the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also + + + + + + + + + + + +
    Property NameDefaultMeaningSince Version
    2.3.0
    spark.sql.orc.mergeSchemafalse +

    + When true, the ORC data source merges schemas collected from all data files, + otherwise the schema is picked from a random data file. +

    +
    3.0.0
    spark.sql.hive.convertMetastoreOrctrue + When set to false, Spark SQL will use the Hive SerDe for ORC tables instead of the built in + support. + 2.0.0
    diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index bd54554baa..8cf1a9c6f7 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,7 +30,7 @@ license: | - In Spark 3.2, `ALTER TABLE .. RENAME TO PARTITION` throws `PartitionAlreadyExistsException` instead of `AnalysisException` for tables from Hive external when the target partition already exists. - - In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`. + - In Spark 3.2, script transform default FIELD DELIMIT is `\u0001` for no serde mode, serde property `field.delim` is `\t` for Hive serde mode when user specifies serde. In Spark 3.1 or earlier, the default FIELD DELIMIT is `\t`, serde property `field.delim` is `\u0001` for Hive serde mode when user specifies serde. ## Upgrading from Spark SQL 3.0 to 3.1 diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 16059a5a08..22f4cf78f5 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -156,6 +156,7 @@ The behavior of some SQL functions can be different under ANSI mode (`spark.sql. - `make_date`: This function should fail with an exception if the result date is invalid. - `make_timestamp`: This function should fail with an exception if the result timestamp is invalid. - `make_interval`: This function should fail with an exception if the result interval is invalid. + - `next_day`: This function throws `IllegalArgumentException` if input is not a valid day of week. ### SQL Operators diff --git a/docs/sql-ref-syntax-qry-select-like.md b/docs/sql-ref-syntax-qry-select-like.md index 6211faa8d5..3604a9ba1e 100644 --- a/docs/sql-ref-syntax-qry-select-like.md +++ b/docs/sql-ref-syntax-qry-select-like.md @@ -21,12 +21,14 @@ license: | ### Description -A LIKE predicate is used to search for a specific pattern. +A LIKE predicate is used to search for a specific pattern. This predicate also supports multiple patterns with quantifiers include `ANY`, `SOME` and `ALL`. ### Syntax ```sql [ NOT ] { LIKE search_pattern [ ESCAPE esc_char ] | [ RLIKE | REGEXP ] regex_pattern } + +[ NOT ] { LIKE quantifiers ( search_pattern [ , ... ]) } ``` ### Parameters @@ -45,6 +47,10 @@ A LIKE predicate is used to search for a specific pattern. * **regex_pattern** Specifies a regular expression search pattern to be searched by the `RLIKE` or `REGEXP` clause. + +* **quantifiers** + + Specifies the predicate quantifiers include `ANY`, `SOME` and `ALL`. `ANY` or `SOME` means if one of the patterns matches the input, then return true; `ALL` means if all the patterns matches the input, then return true. ### Examples @@ -111,6 +117,58 @@ SELECT * FROM person WHERE name LIKE '%$_%' ESCAPE '$'; +---+------+---+ |500|Evan_W| 16| +---+------+---+ + +SELECT * FROM person WHERE name LIKE ALL ('%an%', '%an'); ++---+----+----+ +| id|name| age| ++---+----+----+ +|400| Dan| 50| ++---+----+----+ + +SELECT * FROM person WHERE name LIKE ANY ('%an%', '%an'); ++---+------+---+ +| id| name|age| ++---+------+---+ +|400| Dan| 50| +|500|Evan_W| 16| ++---+------+---+ + +SELECT * FROM person WHERE name LIKE SOME ('%an%', '%an'); ++---+------+---+ +| id| name|age| ++---+------+---+ +|400| Dan| 50| +|500|Evan_W| 16| ++---+------+---+ + +SELECT * FROM person WHERE name NOT LIKE ALL ('%an%', '%an'); ++---+----+----+ +| id|name| age| ++---+----+----+ +|100|John| 30| +|200|Mary|null| +|300|Mike| 80| ++---+----+----+ + +SELECT * FROM person WHERE name NOT LIKE ANY ('%an%', '%an'); ++---+------+----+ +| id| name| age| ++---+------+----+ +|100| John| 30| +|200| Mary|null| +|300| Mike| 80| +|500|Evan_W| 16| ++---+------+----+ + +SELECT * FROM person WHERE name NOT LIKE SOME ('%an%', '%an'); ++---+------+----+ +| id| name| age| ++---+------+----+ +|100| John| 30| +|200| Mary|null| +|300| Mike| 80| +|500|Evan_W| 16| ++---+------+----+ ``` ### Related Statements diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index bac7c2bc6a..5820a5c906 100644 --- a/docs/sql-ref-syntax-qry-select.md +++ b/docs/sql-ref-syntax-qry-select.md @@ -41,7 +41,7 @@ select_statement [ { UNION | INTERSECT | EXCEPT } [ ALL | DISTINCT ] select_stat While `select_statement` is defined as ```sql -SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } +SELECT [ hints , ... ] [ ALL | DISTINCT ] { [ named_expression | regex_column_names ] [ , ... ] } FROM { from_item [ , ... ] } [ PIVOT clause ] [ LATERAL VIEW clause ] [ ... ] @@ -151,6 +151,18 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { named_expression [ , ... ] } Specifies aliases for one or more source window specifications. The source window specifications can be referenced in the widow definitions in the query. + +* **regex_column_names** + + When `spark.sql.parser.quotedRegexColumnNames` is true, quoted identifiers (using backticks) in `SELECT` + statement are interpreted as regular expressions and `SELECT` statement can take regex-based column specification. + For example, below SQL will only take column `c`: + + ```sql + SELECT `(a|b)?+.+` FROM ( + SELECT 1 as a, 2 as b, 3 as c + ) + ``` ### Related Statements diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 5336695478..bf25d46f2e 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -1004,6 +1004,14 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} 3.0.0 + + spark.kafka.clusters.${cluster}.ssl.truststore.type + None + + The file format of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token. + + 3.2.0 + spark.kafka.clusters.${cluster}.ssl.truststore.location None @@ -1021,6 +1029,15 @@ Delegation tokens can be obtained from multiple clusters and ${cluster} 3.0.0 + + spark.kafka.clusters.${cluster}.ssl.keystore.type + None + + The file format of the key store file. This is optional for client. + For further details please see Kafka documentation. Only used to obtain delegation token. + + 3.2.0 + spark.kafka.clusters.${cluster}.ssl.keystore.location None diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 82569653c1..12ebddf72b 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -246,7 +246,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession { dataFileWriter.create(schema, new File(avroFile)) val logicalType = LogicalTypes.decimal(precision, scale) - decimalInputData.map { x => + decimalInputData.foreach { x => val avroRec = new GenericData.Record(schema) val decimal = new java.math.BigDecimal(x).setScale(scale) val bytes = diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala index ed4a6f1e34..21ba7b21ed 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala @@ -31,8 +31,10 @@ private[spark] case class KafkaTokenClusterConf( targetServersRegex: String, securityProtocol: String, kerberosServiceName: String, + trustStoreType: Option[String], trustStoreLocation: Option[String], trustStorePassword: Option[String], + keyStoreType: Option[String], keyStoreLocation: Option[String], keyStorePassword: Option[String], keyPassword: Option[String], @@ -44,8 +46,10 @@ private[spark] case class KafkaTokenClusterConf( s"targetServersRegex=$targetServersRegex, " + s"securityProtocol=$securityProtocol, " + s"kerberosServiceName=$kerberosServiceName, " + + s"trustStoreType=$trustStoreType, " + s"trustStoreLocation=$trustStoreLocation, " + s"trustStorePassword=${trustStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"keyStoreType=$keyStoreType, " + s"keyStoreLocation=$keyStoreLocation, " + s"keyStorePassword=${keyStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + s"keyPassword=${keyPassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + @@ -77,8 +81,10 @@ private [kafka010] object KafkaTokenSparkConf extends Logging { DEFAULT_SECURITY_PROTOCOL_CONFIG), sparkClusterConf.getOrElse(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME), + sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG), sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG), + sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG), sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG), sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG), sparkClusterConf.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG), diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala index bc790418de..a182d3c308 100644 --- a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -162,6 +162,9 @@ private[spark] object KafkaTokenUtil extends Logging { private def setTrustStoreProperties( clusterConf: KafkaTokenClusterConf, properties: ju.Properties): Unit = { + clusterConf.trustStoreType.foreach { truststoreType => + properties.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreType) + } clusterConf.trustStoreLocation.foreach { truststoreLocation => properties.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation) } @@ -173,6 +176,9 @@ private[spark] object KafkaTokenUtil extends Logging { private def setKeyStoreProperties( clusterConf: KafkaTokenClusterConf, properties: ju.Properties): Unit = { + clusterConf.keyStoreType.foreach { keystoreType => + properties.put(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, keystoreType) + } clusterConf.keyStoreLocation.foreach { keystoreLocation => properties.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, keystoreLocation) } @@ -273,8 +279,8 @@ private[spark] object KafkaTokenUtil extends Logging { sparkConf: SparkConf, params: ju.Map[String, Object], clusterConfig: Option[KafkaTokenClusterConf]): Boolean = { - if (HadoopDelegationTokenManager.isServiceEnabled(sparkConf, "kafka") && - clusterConfig.isDefined && params.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) { + if (clusterConfig.isDefined && params.containsKey(SaslConfigs.SASL_JAAS_CONFIG) && + HadoopDelegationTokenManager.isServiceEnabled(sparkConf, "kafka")) { logDebug("Delegation token used by connector, checking if uses the latest token.") val connectorJaasParams = params.get(SaslConfigs.SASL_JAAS_CONFIG).asInstanceOf[String] getTokenJaasParams(clusterConfig.get) != connectorJaasParams diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala index 19335f4221..8271acdc7d 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala @@ -51,8 +51,10 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { protected val bootStrapServers = "127.0.0.1:0" protected val matchingTargetServersRegex = "127.0.0.*:0" protected val nonMatchingTargetServersRegex = "127.0.intentionally_non_matching.*:0" + protected val trustStoreType = "customTrustStoreType" protected val trustStoreLocation = "/path/to/trustStore" protected val trustStorePassword = "trustStoreSecret" + protected val keyStoreType = "customKeyStoreType" protected val keyStoreLocation = "/path/to/keyStore" protected val keyStorePassword = "keyStoreSecret" protected val keyPassword = "keySecret" @@ -124,8 +126,10 @@ trait KafkaDelegationTokenTest extends BeforeAndAfterEach { KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX, securityProtocol, KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME, + Some(trustStoreType), Some(trustStoreLocation), Some(trustStorePassword), + Some(keyStoreType), Some(keyStoreLocation), Some(keyStorePassword), Some(keyPassword), diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala index 61184a6fac..17caf96818 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala @@ -29,8 +29,10 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { private val targetServersRegex = "127.0.0.*:0" private val securityProtocol = SSL.name private val kerberosServiceName = "kafka1" + private val trustStoreType = "customTrustStoreType" private val trustStoreLocation = "/path/to/trustStore" private val trustStorePassword = "trustStoreSecret" + private val keyStoreType = "customKeyStoreType" private val keyStoreLocation = "/path/to/keyStore" private val keyStorePassword = "keyStoreSecret" private val keyPassword = "keySecret" @@ -60,8 +62,10 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { assert(clusterConfig.securityProtocol === SASL_SSL.name) assert(clusterConfig.kerberosServiceName === KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME) + assert(clusterConfig.trustStoreType === None) assert(clusterConfig.trustStoreLocation === None) assert(clusterConfig.trustStorePassword === None) + assert(clusterConfig.keyStoreType === None) assert(clusterConfig.keyStoreLocation === None) assert(clusterConfig.keyStorePassword === None) assert(clusterConfig.keyPassword === None) @@ -75,8 +79,10 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { sparkConf.set(s"spark.kafka.clusters.$identifier1.security.protocol", securityProtocol) sparkConf.set(s"spark.kafka.clusters.$identifier1.sasl.kerberos.service.name", kerberosServiceName) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.truststore.type", trustStoreType) sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.truststore.location", trustStoreLocation) sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.truststore.password", trustStorePassword) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.keystore.type", keyStoreType) sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.keystore.location", keyStoreLocation) sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.keystore.password", keyStorePassword) sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.key.password", keyPassword) @@ -88,8 +94,10 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { assert(clusterConfig.targetServersRegex === targetServersRegex) assert(clusterConfig.securityProtocol === securityProtocol) assert(clusterConfig.kerberosServiceName === kerberosServiceName) + assert(clusterConfig.trustStoreType === Some(trustStoreType)) assert(clusterConfig.trustStoreLocation === Some(trustStoreLocation)) assert(clusterConfig.trustStorePassword === Some(trustStorePassword)) + assert(clusterConfig.keyStoreType === Some(keyStoreType)) assert(clusterConfig.keyStoreLocation === Some(keyStoreLocation)) assert(clusterConfig.keyStorePassword === Some(keyStorePassword)) assert(clusterConfig.keyPassword === Some(keyPassword)) @@ -127,8 +135,10 @@ class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { assert(clusterConfig.securityProtocol === SASL_SSL.name) assert(clusterConfig.kerberosServiceName === KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME) + assert(clusterConfig.trustStoreType === None) assert(clusterConfig.trustStoreLocation === None) assert(clusterConfig.trustStorePassword === None) + assert(clusterConfig.keyStoreType === None) assert(clusterConfig.keyStoreLocation === None) assert(clusterConfig.keyStorePassword === None) assert(clusterConfig.keyPassword === None) diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala index 94f7853003..ca34e14f2c 100644 --- a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala @@ -64,8 +64,10 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { === bootStrapServers) assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) === SASL_PLAINTEXT.name) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEY_PASSWORD_CONFIG)) @@ -80,10 +82,12 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { === bootStrapServers) assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) === SASL_SSL.name) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG) === trustStoreType) assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG) === trustStoreLocation) assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) === trustStorePassword) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG)) assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEY_PASSWORD_CONFIG)) @@ -99,10 +103,12 @@ class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { === bootStrapServers) assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) === SSL.name) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG) === trustStoreType) assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG) === trustStoreLocation) assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) === trustStorePassword) + assert(adminClientProperties.get(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG) === keyStoreType) assert(adminClientProperties.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG) === keyStoreLocation) assert(adminClientProperties.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) === keyStorePassword) assert(adminClientProperties.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG) === keyPassword) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 47dac3ec29..c20674718b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -107,9 +107,9 @@ class NaiveBayesSuite extends SparkFunSuite with MLlibTestSparkContext { val modelIndex = piData.indices.zip(model.labels.map(_.toInt)) try { for (i <- modelIndex) { - assert(math.exp(piData(i._2)) ~== math.exp(model.pi(i._1)) absTol 0.05) + assert(piData(i._2) ~== model.pi(i._1) relTol 0.15) for (j <- thetaData(i._2).indices) { - assert(math.exp(thetaData(i._2)(j)) ~== math.exp(model.theta(i._1)(j)) absTol 0.05) + assert(thetaData(i._2)(j) ~== model.theta(i._1)(j) relTol 0.15) } } } catch { diff --git a/pom.xml b/pom.xml index 91ca0398a0..f921e35a76 100644 --- a/pom.xml +++ b/pom.xml @@ -134,7 +134,8 @@ 2.3 2.6.0 - 10.12.1.1 + + 10.14.2.0 1.10.1 1.6.6 9.4.34.v20201102 @@ -2468,7 +2469,7 @@ net.alchim31.maven scala-maven-plugin - 4.4.0 + 4.3.0 eclipse-add-source @@ -3371,6 +3372,12 @@ org.openlabtesting.leveldbjni + + + linux + aarch64 + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa3e2cd65e..668701be0a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -958,18 +958,24 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("org/apache/hadoop"))) }, - javacOptions in (JavaUnidoc, unidoc) := Seq( - "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", - "-public", - "-noqualifier", "java.lang", - "-tag", """example:a:Example\:""", - "-tag", """note:a:Note\:""", - "-tag", "group:X", - "-tag", "tparam:X", - "-tag", "constructor:X", - "-tag", "todo:X", - "-tag", "groupname:X" - ), + javacOptions in (JavaUnidoc, unidoc) := { + val versionParts = System.getProperty("java.version").split("[+.\\-]+", 3) + var major = versionParts(0).toInt + if (major == 1) major = versionParts(1).toInt + + Seq( + "-windowtitle", "Spark " + version.value.replaceAll("-SNAPSHOT", "") + " JavaDoc", + "-public", + "-noqualifier", "java.lang", + "-tag", """example:a:Example\:""", + "-tag", """note:a:Note\:""", + "-tag", "group:X", + "-tag", "tparam:X", + "-tag", "constructor:X", + "-tag", "todo:X", + "-tag", "groupname:X", + ) ++ { if (major >= 9) Seq("--ignore-source-errors", "-notree") else Seq.empty } + }, // Use GitHub repository for Scaladoc source links unidocSourceBase := s"https://github.com/apache/spark/tree/v${version.value}", diff --git a/python/docs/source/getting_started/index.rst b/python/docs/source/getting_started/index.rst index 9fa3352ae2..38b9c935fc 100644 --- a/python/docs/source/getting_started/index.rst +++ b/python/docs/source/getting_started/index.rst @@ -21,6 +21,9 @@ Getting Started =============== This page summarizes the basic steps required to setup and get started with PySpark. +There are more guides shared with other languages such as +`Quick Start `_ in Programming Guides +at `the Spark documentation `_. .. toctree:: :maxdepth: 2 diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index a90f5fe159..c5485424da 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -48,11 +48,11 @@ If you want to install extra dependencies for a specific component, you can inst pip install pyspark[sql] -For PySpark with/without a specific Hadoop version, you can install it by using ``HADOOP_VERSION`` environment variables as below: +For PySpark with/without a specific Hadoop version, you can install it by using ``PYSPARK_HADOOP_VERSION`` environment variables as below: .. code-block:: bash - HADOOP_VERSION=2.7 pip install pyspark + PYSPARK_HADOOP_VERSION=2.7 pip install pyspark The default distribution uses Hadoop 3.2 and Hive 2.3. If users specify different versions of Hadoop, the pip installation automatically downloads a different version and use it in PySpark. Downloading it can take a while depending on @@ -60,15 +60,15 @@ the network and the mirror chosen. ``PYSPARK_RELEASE_MIRROR`` can be set to manu .. code-block:: bash - PYSPARK_RELEASE_MIRROR=http://mirror.apache-kr.org HADOOP_VERSION=2.7 pip install + PYSPARK_RELEASE_MIRROR=http://mirror.apache-kr.org PYSPARK_HADOOP_VERSION=2.7 pip install It is recommended to use ``-v`` option in ``pip`` to track the installation and download status. .. code-block:: bash - HADOOP_VERSION=2.7 pip install pyspark -v + PYSPARK_HADOOP_VERSION=2.7 pip install pyspark -v -Supported values in ``HADOOP_VERSION`` are: +Supported values in ``PYSPARK_HADOOP_VERSION`` are: - ``without``: Spark pre-built with user-provided Apache Hadoop - ``2.7``: Spark pre-built for Apache Hadoop 2.7 diff --git a/python/docs/source/migration_guide/index.rst b/python/docs/source/migration_guide/index.rst index 41e36b16b3..88e768dc46 100644 --- a/python/docs/source/migration_guide/index.rst +++ b/python/docs/source/migration_guide/index.rst @@ -21,8 +21,6 @@ Migration Guide =============== This page describes the migration guide specific to PySpark. -Many items of other migration guides can also be applied when migrating PySpark to higher versions because PySpark internally shares other components. -Please also refer other migration guides such as `Migration Guide: SQL, Datasets and DataFrame `_. .. toctree:: :maxdepth: 2 @@ -33,3 +31,13 @@ Please also refer other migration guides such as `Migration Guide: SQL, Datasets pyspark_2.2_to_2.3 pyspark_1.4_to_1.5 pyspark_1.0_1.2_to_1.3 + + +Many items of other migration guides can also be applied when migrating PySpark to higher versions because PySpark internally shares other components. +Please also refer other migration guides: + +- `Migration Guide: Spark Core `_ +- `Migration Guide: SQL, Datasets and DataFrame `_ +- `Migration Guide: Structured Streaming `_ +- `Migration Guide: MLlib (Machine Learning) `_ + diff --git a/python/docs/source/reference/pyspark.ml.rst b/python/docs/source/reference/pyspark.ml.rst index 2de0ff65a3..cc904597d2 100644 --- a/python/docs/source/reference/pyspark.ml.rst +++ b/python/docs/source/reference/pyspark.ml.rst @@ -16,11 +16,11 @@ under the License. -ML -== +MLlib (DataFrame-based) +======================= -ML Pipeline APIs ----------------- +Pipeline APIs +------------- .. currentmodule:: pyspark.ml @@ -188,8 +188,8 @@ Clustering PowerIterationClustering -ML Functions ----------------------------- +Functions +--------- .. currentmodule:: pyspark.ml.functions diff --git a/python/docs/source/reference/pyspark.mllib.rst b/python/docs/source/reference/pyspark.mllib.rst index df5ea017d0..12fc4798dd 100644 --- a/python/docs/source/reference/pyspark.mllib.rst +++ b/python/docs/source/reference/pyspark.mllib.rst @@ -16,8 +16,8 @@ under the License. -MLlib -===== +MLlib (RDD-based) +================= Classification -------------- diff --git a/python/docs/source/user_guide/index.rst b/python/docs/source/user_guide/index.rst index 3e535ce16b..704156b11d 100644 --- a/python/docs/source/user_guide/index.rst +++ b/python/docs/source/user_guide/index.rst @@ -20,9 +20,21 @@ User Guide ========== +This page is the guide for PySpark users which contains PySpark specific topics. + .. toctree:: :maxdepth: 2 arrow_pandas python_packaging + +There are more guides shared with other languages in Programming Guides +at `the Spark documentation `_. + +- `RDD Programming Guide `_ +- `Spark SQL, DataFrames and Datasets Guide `_ +- `Structured Streaming Programming Guide `_ +- `Spark Streaming Programming Guide `_ +- `Machine Learning Library (MLlib) Guide `_ + diff --git a/python/pyspark/find_spark_home.py b/python/pyspark/find_spark_home.py index 4521a36503..62a36d42eb 100755 --- a/python/pyspark/find_spark_home.py +++ b/python/pyspark/find_spark_home.py @@ -36,7 +36,7 @@ def is_spark_home(path): (os.path.isdir(os.path.join(path, "jars")) or os.path.isdir(os.path.join(path, "assembly")))) - # Spark distribution can be downloaded when HADOOP_VERSION environment variable is set. + # Spark distribution can be downloaded when PYSPARK_HADOOP_VERSION environment variable is set. # We should look up this directory first, see also SPARK-32017. spark_dist_dir = "spark-distribution" paths = [ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index f612d2d036..c9d24dc668 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -4355,7 +4355,7 @@ def filter(col, f): return _invoke_higher_order_function("ArrayFilter", [col], [f]) -def aggregate(col, zero, merge, finish=None): +def aggregate(col, initialValue, merge, finish=None): """ Applies a binary operator to an initial state and all elements in the array, and reduces this to a single state. The final state is converted into the final result @@ -4372,7 +4372,7 @@ def aggregate(col, zero, merge, finish=None): ---------- col : :class:`Column` or str name of column or expression - zero : :class:`Column` or str + initialValue : :class:`Column` or str initial value. Name of column or expression merge : function a binary function ``(acc: Column, x: Column) -> Column...`` returning expression @@ -4416,19 +4416,19 @@ def aggregate(col, zero, merge, finish=None): if finish is not None: return _invoke_higher_order_function( "ArrayAggregate", - [col, zero], + [col, initialValue], [merge, finish] ) else: return _invoke_higher_order_function( "ArrayAggregate", - [col, zero], + [col, initialValue], [merge] ) -def zip_with(col1, col2, f): +def zip_with(left, right, f): """ Merge two given arrays, element-wise, into a single array using a function. If one array is shorter, nulls are appended at the end to match the length of the longer @@ -4438,9 +4438,9 @@ def zip_with(col1, col2, f): Parameters ---------- - col1 : :class:`Column` or str + left : :class:`Column` or str name of the first column or expression - col2 : :class:`Column` or str + right : :class:`Column` or str name of the second column or expression f : function a binary function ``(x1: Column, x2: Column) -> Column...`` @@ -4471,7 +4471,7 @@ def zip_with(col1, col2, f): |[foo_1, bar_2, 3]| +-----------------+ """ - return _invoke_higher_order_function("ZipWith", [col1, col2], [f]) + return _invoke_higher_order_function("ZipWith", [left, right], [f]) def transform_keys(col, f): diff --git a/python/pyspark/sql/functions.pyi b/python/pyspark/sql/functions.pyi index acb17a2657..0cf60c0c26 100644 --- a/python/pyspark/sql/functions.pyi +++ b/python/pyspark/sql/functions.pyi @@ -237,13 +237,13 @@ def filter(col: ColumnOrName, f: Callable[[Column], Column]) -> Column: ... def filter(col: ColumnOrName, f: Callable[[Column, Column], Column]) -> Column: ... def aggregate( col: ColumnOrName, - zero: ColumnOrName, + initialValue: ColumnOrName, merge: Callable[[Column, Column], Column], finish: Optional[Callable[[Column], Column]] = ..., ) -> Column: ... def zip_with( - col1: ColumnOrName, - ColumnOrName: ColumnOrName, + left: ColumnOrName, + right: ColumnOrName, f: Callable[[Column, Column], Column], ) -> Column: ... def transform_keys( diff --git a/python/setup.py b/python/setup.py index 7bb8a00171..c7f195b89a 100644 --- a/python/setup.py +++ b/python/setup.py @@ -125,16 +125,16 @@ def run(self): spark_dist = os.path.join(self.install_lib, "pyspark", "spark-distribution") rmtree(spark_dist, ignore_errors=True) - if ("HADOOP_VERSION" in os.environ) or ("HIVE_VERSION" in os.environ): - # Note that SPARK_VERSION environment is just a testing purpose. - # HIVE_VERSION environment variable is also internal for now in case + if ("PYSPARK_HADOOP_VERSION" in os.environ) or ("PYSPARK_HIVE_VERSION" in os.environ): + # Note that PYSPARK_VERSION environment is just a testing purpose. + # PYSPARK_HIVE_VERSION environment variable is also internal for now in case # we support another version of Hive in the future. spark_version, hadoop_version, hive_version = install_module.checked_versions( - os.environ.get("SPARK_VERSION", VERSION).lower(), - os.environ.get("HADOOP_VERSION", install_module.DEFAULT_HADOOP).lower(), - os.environ.get("HIVE_VERSION", install_module.DEFAULT_HIVE).lower()) + os.environ.get("PYSPARK_VERSION", VERSION).lower(), + os.environ.get("PYSPARK_HADOOP_VERSION", install_module.DEFAULT_HADOOP).lower(), + os.environ.get("PYSPARK_HIVE_VERSION", install_module.DEFAULT_HIVE).lower()) - if ("SPARK_VERSION" not in os.environ and + if ("PYSPARK_VERSION" not in os.environ and ((install_module.DEFAULT_HADOOP, install_module.DEFAULT_HIVE) == (hadoop_version, hive_version))): # Do not download and install if they are same as default. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 6939de4697..8dca875b54 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -99,6 +99,14 @@ private[spark] object Config extends Logging { .toSequence .createWithDefault(Nil) + val CONFIG_MAP_MAXSIZE = + ConfigBuilder("spark.kubernetes.configMap.maxSize") + .doc("Max size limit for a config map. This is configurable as per" + + " https://etcd.io/docs/v3.4.0/dev-guide/limit/ on k8s server end.") + .version("3.1.0") + .longConf + .createWithDefault(1572864) // 1.5 MiB + val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" val KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX = "spark.kubernetes.authenticate.executor" val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala index 32f630f77d..4207077677 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala @@ -18,15 +18,17 @@ package org.apache.spark.deploy.k8s.submit import java.io.{File, StringWriter} +import java.nio.charset.MalformedInputException import java.util.Properties import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.io.{Codec, Source} import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, KeyToPath} import org.apache.spark.SparkConf -import org.apache.spark.deploy.k8s.{Constants, KubernetesUtils} +import org.apache.spark.deploy.k8s.{Config, Constants, KubernetesUtils} import org.apache.spark.deploy.k8s.Constants.ENV_SPARK_CONF_DIR import org.apache.spark.internal.Logging @@ -54,8 +56,10 @@ private[spark] object KubernetesClientUtils extends Logging { /** * Build, file -> 'file's content' map of all the selected files in SPARK_CONF_DIR. */ - def buildSparkConfDirFilesMap(configMapName: String, - sparkConf: SparkConf, resolvedPropertiesMap: Map[String, String]): Map[String, String] = { + def buildSparkConfDirFilesMap( + configMapName: String, + sparkConf: SparkConf, + resolvedPropertiesMap: Map[String, String]): Map[String, String] = synchronized { val loadedConfFilesMap = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) // Add resolved spark conf to the loaded configuration files map. if (resolvedPropertiesMap.nonEmpty) { @@ -90,29 +94,71 @@ private[spark] object KubernetesClientUtils extends Logging { .build() } - private def loadSparkConfDirFiles(conf: SparkConf): Map[String, String] = { + private def orderFilesBySize(confFiles: Seq[File]): Seq[File] = { + val fileToFileSizePairs = confFiles.map(f => (f, f.getName.length + f.length())) + // sort first by name and then by length, so that during tests we have consistent results. + fileToFileSizePairs.sortBy(f => f._1).sortBy(f => f._2).map(_._1) + } + + // exposed for testing + private[submit] def loadSparkConfDirFiles(conf: SparkConf): Map[String, String] = { val confDir = Option(conf.getenv(ENV_SPARK_CONF_DIR)).orElse( conf.getOption("spark.home").map(dir => s"$dir/conf")) + val maxSize = conf.get(Config.CONFIG_MAP_MAXSIZE) if (confDir.isDefined) { - val confFiles = listConfFiles(confDir.get) - logInfo(s"Spark configuration files loaded from $confDir : ${confFiles.mkString(",")}") - confFiles.map { file => - val source = Source.fromFile(file)(Codec.UTF8) - val mapping = (file.getName -> source.mkString) - source.close() - mapping - }.toMap + val confFiles: Seq[File] = listConfFiles(confDir.get, maxSize) + val orderedConfFiles = orderFilesBySize(confFiles) + var truncatedMapSize: Long = 0 + val truncatedMap = mutable.HashMap[String, String]() + val skippedFiles = mutable.HashSet[String]() + var source: Source = Source.fromString("") // init with empty source. + for (file <- orderedConfFiles) { + try { + source = Source.fromFile(file)(Codec.UTF8) + val (fileName, fileContent) = file.getName -> source.mkString + if ((truncatedMapSize + fileName.length + fileContent.length) < maxSize) { + truncatedMap.put(fileName, fileContent) + truncatedMapSize = truncatedMapSize + (fileName.length + fileContent.length) + } else { + skippedFiles.add(fileName) + } + } catch { + case e: MalformedInputException => + logWarning( + s"Unable to read a non UTF-8 encoded file ${file.getAbsolutePath}. Skipping...", e) + None + } finally { + source.close() + } + } + if (truncatedMap.nonEmpty) { + logInfo(s"Spark configuration files loaded from $confDir :" + + s" ${truncatedMap.keys.mkString(",")}") + } + if (skippedFiles.nonEmpty) { + logWarning(s"Skipped conf file(s) ${skippedFiles.mkString(",")}, due to size constraint." + + s" Please see, config: `${Config.CONFIG_MAP_MAXSIZE.key}` for more details.") + } + truncatedMap.toMap } else { Map.empty[String, String] } } - private def listConfFiles(confDir: String): Seq[File] = { - // We exclude all the template files and user provided spark conf or properties. - // As spark properties are resolved in a different step. + private def listConfFiles(confDir: String, maxSize: Long): Seq[File] = { + // At the moment configmaps do not support storing binary content (i.e. skip jar,tar,gzip,zip), + // and configMaps do not allow for size greater than 1.5 MiB(configurable). + // https://etcd.io/docs/v3.4.0/dev-guide/limit/ + def testIfTooLargeOrBinary(f: File): Boolean = (f.length() + f.getName.length > maxSize) || + f.getName.matches(".*\\.(gz|zip|jar|tar)") + + // We exclude all the template files and user provided spark conf or properties, + // Spark properties are resolved in a different step. + def testIfSparkConfOrTemplates(f: File) = f.getName.matches(".*\\.template") || + f.getName.matches("spark.*(conf|properties)") + val fileFilter = (f: File) => { - f.isFile && !(f.getName.endsWith("template") || - f.getName.matches("spark.*(conf|properties)")) + f.isFile && !testIfTooLargeOrBinary(f) && !testIfSparkConfOrTemplates(f) } val confFiles: Seq[File] = { val dir = new File(confDir) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala index 71355c7af1..cb4d8810e5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshot.scala @@ -41,7 +41,7 @@ private[spark] case class ExecutorPodsSnapshot(executorPods: Map[Long, ExecutorP object ExecutorPodsSnapshot extends Logging { private var shouldCheckAllContainers: Boolean = _ - private var sparkContainerName: String = _ + private var sparkContainerName: String = DEFAULT_EXECUTOR_CONTAINER_NAME def apply(executorPods: Seq[Pod]): ExecutorPodsSnapshot = { ExecutorPodsSnapshot(toStatesByExecutorId(executorPods)) @@ -80,23 +80,21 @@ object ExecutorPodsSnapshot extends Logging { .anyMatch(t => t != null && t.getExitCode != 0)) { PodFailed(pod) } else { - // Otherwise look for the Spark container - val sparkContainerStatusOpt = pod.getStatus.getContainerStatuses.asScala - .find(_.getName() == sparkContainerName) - sparkContainerStatusOpt match { - case Some(sparkContainerStatus) => - sparkContainerStatus.getState.getTerminated match { - case t if t.getExitCode != 0 => - PodFailed(pod) - case t if t.getExitCode == 0 => + // Otherwise look for the Spark container and get the exit code if present. + val sparkContainerExitCode = pod.getStatus.getContainerStatuses.asScala + .find(_.getName() == sparkContainerName).flatMap(x => Option(x.getState)) + .flatMap(x => Option(x.getTerminated)).flatMap(x => Option(x.getExitCode)) + .map(_.toInt) + sparkContainerExitCode match { + case Some(t) => + t match { + case 0 => PodSucceeded(pod) case _ => - PodRunning(pod) + PodFailed(pod) } - // If we can't find the Spark container status, fall back to the pod status + // No exit code means we are running. case _ => - logWarning(s"Unable to find container ${sparkContainerName} in pod ${pod} " + - "defaulting to entire pod status (running).") PodRunning(pod) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 1a14d52400..18d0c00edf 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -191,25 +191,32 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { assert(configMap.getData.get(SPARK_CONF_FILE_NAME).contains("conf2key=conf2value")) } - test("All files from SPARK_CONF_DIR, except templates and spark config " + + test("All files from SPARK_CONF_DIR, " + + "except templates, spark config, binary files and are within size limit, " + "should be populated to pod's configMap.") { def testSetup: (SparkConf, Seq[String]) = { val tempDir = Utils.createTempDir() - val sparkConf = new SparkConf(loadDefaults = false).setSparkHome(tempDir.getAbsolutePath) + val sparkConf = new SparkConf(loadDefaults = false) + .setSparkHome(tempDir.getAbsolutePath) val tempConfDir = new File(s"${tempDir.getAbsolutePath}/conf") tempConfDir.mkdir() // File names - which should not get mounted on the resultant config map. val filteredConfFileNames = - Set("spark-env.sh.template", "spark.properties", "spark-defaults.conf") - val confFileNames = for (i <- 1 to 5) yield s"testConf.$i" ++ + Set("spark-env.sh.template", "spark.properties", "spark-defaults.conf", + "test.gz", "test2.jar", "non_utf8.txt") + val confFileNames = (for (i <- 1 to 5) yield s"testConf.$i") ++ List("spark-env.sh") ++ filteredConfFileNames - val testConfFiles = for (i <- confFileNames) yield { + val testConfFiles = (for (i <- confFileNames) yield { val file = new File(s"${tempConfDir.getAbsolutePath}/$i") - Files.write(file.toPath, "conf1key=conf1value".getBytes(StandardCharsets.UTF_8)) + if (i.startsWith("non_utf8")) { // filling some non-utf-8 binary + Files.write(file.toPath, Array[Byte](0x00.toByte, 0xA1.toByte)) + } else { + Files.write(file.toPath, "conf1key=conf1value".getBytes(StandardCharsets.UTF_8)) + } file.getName - } + }) assert(tempConfDir.listFiles().length == confFileNames.length) val expectedConfFiles: Seq[String] = testConfFiles.filterNot(filteredConfFileNames.contains) (sparkConf, expectedConfFiles) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala new file mode 100644 index 0000000000..ee672cc041 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtilsSuite.scala @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.deploy.k8s.submit + +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config +import org.apache.spark.util.Utils + +class KubernetesClientUtilsSuite extends SparkFunSuite with BeforeAndAfter { + + def testSetup(inputFiles: Map[String, Array[Byte]]): SparkConf = { + val tempDir = Utils.createTempDir() + val sparkConf = new SparkConf(loadDefaults = false) + .setSparkHome(tempDir.getAbsolutePath) + + val tempConfDir = new File(s"${tempDir.getAbsolutePath}/conf") + tempConfDir.mkdir() + for (i <- inputFiles) yield { + val file = new File(s"${tempConfDir.getAbsolutePath}/${i._1}") + Files.write(file.toPath, i._2) + file.getName + } + sparkConf + } + + test("verify load files, loads only allowed files and not the disallowed files.") { + val input: Map[String, Array[Byte]] = Map("test.txt" -> "test123", "z12.zip" -> "zZ", + "rere.jar" -> "@31", "spark.jar" -> "@31", "_test" -> "", "sample.conf" -> "conf") + .map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8)) ++ + Map("binary-file.conf" -> Array[Byte](0x00.toByte, 0xA1.toByte)) + val sparkConf = testSetup(input) + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("test.txt" -> "test123", "sample.conf" -> "conf", "_test" -> "") + assert(output === expectedOutput) + } + + test("verify load files, truncates the content to maxSize, when keys are very large in number.") { + val input = (for (i <- 10000 to 1 by -1) yield (s"testConf.${i}" -> "test123456")).toMap + val sparkConf = testSetup(input.map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8))) + .set(Config.CONFIG_MAP_MAXSIZE.key, "60") + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("testConf.1" -> "test123456", "testConf.2" -> "test123456") + assert(output === expectedOutput) + val output1 = KubernetesClientUtils.loadSparkConfDirFiles( + sparkConf.set(Config.CONFIG_MAP_MAXSIZE.key, "250000")) + assert(output1 === input) + } + + test("verify load files, truncates the content to maxSize, when keys are equal in length.") { + val input = (for (i <- 9 to 1 by -1) yield (s"testConf.${i}" -> "test123456")).toMap + val sparkConf = testSetup(input.map(f => f._1 -> f._2.getBytes(StandardCharsets.UTF_8))) + .set(Config.CONFIG_MAP_MAXSIZE.key, "80") + val output = KubernetesClientUtils.loadSparkConfDirFiles(sparkConf) + val expectedOutput = Map("testConf.1" -> "test123456", "testConf.2" -> "test123456", + "testConf.3" -> "test123456") + assert(output === expectedOutput) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index 225278c2aa..41cba573d8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -115,13 +115,16 @@ object ExecutorLifecycleTestUtils { .editOrNewStatus() .withPhase("running") .addNewContainerStatus() + .withName(DEFAULT_EXECUTOR_CONTAINER_NAME) .withNewState() .withNewTerminated() + .withMessage("message") .withExitCode(exitCode) .endTerminated() .endState() .endContainerStatus() .addNewContainerStatus() + .withName("SIDECARFRIEND") .withNewState() .withNewRunning() .endRunning() diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index a5a2611be3..e9fbff730f 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -138,7 +138,7 @@ private[spark] object MesosSchedulerBackendUtil extends Logging { val containerInfo = ContainerInfo.newBuilder() .setType(containerType) - conf.get(EXECUTOR_DOCKER_IMAGE).map { image => + conf.get(EXECUTOR_DOCKER_IMAGE).foreach { image => val forcePullImage = conf .get(EXECUTOR_DOCKER_FORCE_PULL_IMAGE).contains(true) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index ac50c1c77a..f236fc39f6 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -263,7 +263,7 @@ private[yarn] class YarnAllocator( private def getPendingAtLocation( location: String): Map[Int, Seq[ContainerRequest]] = synchronized { val allContainerRequests = new mutable.HashMap[Int, Seq[ContainerRequest]] - rpIdToResourceProfile.keys.map { id => + rpIdToResourceProfile.keys.foreach { id => val profResource = rpIdToYarnResource.get(id) val result = amClient.getMatchingRequests(getContainerPriority(id), location, profResource) .asScala.flatMap(_.asScala) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala index ff4c54df96..ed18e94f46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryCompilationErrors.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.errors +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{ResolvedNamespace, ResolvedView} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SerdeInfo} @@ -364,10 +366,16 @@ object QueryCompilationErrors { new AnalysisException(s"undefined function $name") } + def invalidFunctionArgumentsError( + name: String, expectedInfo: String, actualNumber: Int): Throwable = { + new AnalysisException(s"Invalid number of arguments for function $name. " + + s"Expected: $expectedInfo; Found: $actualNumber") + } + def invalidFunctionArgumentNumberError( validParametersCount: Seq[Int], name: String, params: Seq[Class[Expression]]): Throwable = { - val invalidArgumentsMsg = if (validParametersCount.length == 0) { - s"Invalid arguments for function $name" + if (validParametersCount.length == 0) { + new AnalysisException(s"Invalid arguments for function $name") } else { val expectedNumberOfParameters = if (validParametersCount.length == 1) { validParametersCount.head.toString @@ -375,10 +383,8 @@ object QueryCompilationErrors { validParametersCount.init.mkString("one of ", ", ", " and ") + validParametersCount.last } - s"Invalid number of arguments for function $name. " + - s"Expected: $expectedNumberOfParameters; Found: ${params.length}" + invalidFunctionArgumentsError(name, expectedNumberOfParameters, params.length) } - new AnalysisException(invalidArgumentsMsg) } def functionAcceptsOnlyOneArgumentError(name: String): Throwable = { @@ -504,4 +510,156 @@ object QueryCompilationErrors { def columnDoesNotExistError(colName: String): Throwable = { new AnalysisException(s"Column $colName does not exist") } + + def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = { + new AnalysisException( + s"rename temporary view from '$oldName' to '$newName': destination view already exists") + } + + def databaseNotEmptyError(db: String, details: String): Throwable = { + new AnalysisException(s"Database $db is not empty. One or more $details exist.") + } + + def invalidNameForTableOrDatabaseError(name: String): Throwable = { + new AnalysisException(s"`$name` is not a valid name for tables/databases. " + + "Valid names only contain alphabet characters, numbers and _.") + } + + def cannotCreateDatabaseWithSameNameAsPreservedDatabaseError(database: String): Throwable = { + new AnalysisException(s"$database is a system preserved database, " + + "you cannot create a database with this name.") + } + + def cannotDropDefaultDatabaseError(): Throwable = { + new AnalysisException("Can not drop default database") + } + + def cannotUsePreservedDatabaseAsCurrentDatabaseError(database: String): Throwable = { + new AnalysisException(s"$database is a system preserved database, you cannot use it as " + + "current database. To access global temporary views, you should use qualified name with " + + s"the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM $database.viewName.") + } + + def createExternalTableWithoutLocationError(): Throwable = { + new AnalysisException("CREATE EXTERNAL TABLE must be accompanied by LOCATION") + } + + def cannotOperateManagedTableWithExistingLocationError( + methodName: String, tableIdentifier: TableIdentifier, tableLocation: Path): Throwable = { + new AnalysisException(s"Can not $methodName the managed table('$tableIdentifier')" + + s". The associated location('${tableLocation.toString}') already exists.") + } + + def dropNonExistentColumnsNotSupportedError( + nonExistentColumnNames: Seq[String]): Throwable = { + new AnalysisException( + s""" + |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are + |not present in the new schema. We don't support dropping columns yet. + """.stripMargin) + } + + def cannotRetrieveTableOrViewNotInSameDatabaseError( + qualifiedTableNames: Seq[QualifiedTableName]): Throwable = { + new AnalysisException("Only the tables/views belong to the same database can be retrieved. " + + s"Querying tables/views are $qualifiedTableNames") + } + + def renameTableSourceAndDestinationMismatchError(db: String, newDb: String): Throwable = { + new AnalysisException( + s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + } + + def cannotRenameTempViewWithDatabaseSpecifiedError( + oldName: TableIdentifier, newName: TableIdentifier): Throwable = { + new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': cannot " + + s"specify database name '${newName.database.get}' in the destination table") + } + + def cannotRenameTempViewToExistingTableError( + oldName: TableIdentifier, newName: TableIdentifier): Throwable = { + new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " + + "destination table already exists") + } + + def invalidPartitionSpecError(details: String): Throwable = { + new AnalysisException(s"Partition spec is invalid. $details") + } + + def functionAlreadyExistsError(func: FunctionIdentifier): Throwable = { + new AnalysisException(s"Function $func already exists") + } + + def cannotLoadClassWhenRegisteringFunctionError( + className: String, func: FunctionIdentifier): Throwable = { + new AnalysisException(s"Can not load class '$className' when registering " + + s"the function '$func', please make sure it is on the classpath") + } + + def v2CatalogNotSupportFunctionError( + catalog: String, namespace: Seq[String]): Throwable = { + new AnalysisException("V2 catalog does not support functions yet. " + + s"catalog: $catalog, namespace: '${namespace.quoted}'") + } + + def resourceTypeNotSupportedError(resourceType: String): Throwable = { + new AnalysisException(s"Resource Type '$resourceType' is not supported.") + } + + def tableNotSpecifyDatabaseError(identifier: TableIdentifier): Throwable = { + new AnalysisException(s"table $identifier did not specify database") + } + + def tableNotSpecifyLocationUriError(identifier: TableIdentifier): Throwable = { + new AnalysisException(s"table $identifier did not specify locationUri") + } + + def partitionNotSpecifyLocationUriError(specString: String): Throwable = { + new AnalysisException(s"Partition [$specString] did not specify locationUri") + } + + def invalidBucketNumberError(bucketingMaxBuckets: Int, numBuckets: Int): Throwable = { + new AnalysisException( + s"Number of buckets should be greater than 0 but less than or equal to " + + s"bucketing.maxBuckets (`$bucketingMaxBuckets`). Got `$numBuckets`") + } + + def corruptedTableNameContextInCatalogError(numParts: Int, index: Int): Throwable = { + new AnalysisException("Corrupted table name context in catalog: " + + s"$numParts parts expected, but part $index is missing.") + } + + def corruptedViewSQLConfigsInCatalogError(e: Exception): Throwable = { + new AnalysisException("Corrupted view SQL configs in catalog", cause = Some(e)) + } + + def corruptedViewQueryOutputColumnsInCatalogError(numCols: String, index: Int): Throwable = { + new AnalysisException("Corrupted view query output column names in catalog: " + + s"$numCols parts expected, but part $index is missing.") + } + + def corruptedViewReferredTempViewInCatalogError(e: Exception): Throwable = { + new AnalysisException("corrupted view referred temp view names in catalog", cause = Some(e)) + } + + def corruptedViewReferredTempFunctionsInCatalogError(e: Exception): Throwable = { + new AnalysisException( + "corrupted view referred temp functions names in catalog", cause = Some(e)) + } + + def columnStatisticsDeserializationNotSupportedError( + name: String, dataType: DataType): Throwable = { + new AnalysisException("Column statistics deserialization is not supported for " + + s"column $name of data type: $dataType.") + } + + def columnStatisticsSerializationNotSupportedError( + colName: String, dataType: DataType): Throwable = { + new AnalysisException("Column statistics serialization is not supported for " + + s"column $colName of data type: $dataType.") + } + + def cannotReadCorruptedTablePropertyError(key: String, details: String = ""): Throwable = { + new AnalysisException(s"Cannot read table property '$key' as it's corrupted.$details") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala index d24e61c699..61dcddb979 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/QueryExecutionErrors.scala @@ -17,7 +17,13 @@ package org.apache.spark.sql.errors +import java.io.IOException + +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.analysis.UnresolvedGenerator +import org.apache.spark.sql.catalyst.catalog.CatalogDatabase import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} /** @@ -56,4 +62,54 @@ object QueryExecutionErrors { def cannotTerminateGeneratorError(generator: UnresolvedGenerator): Throwable = { new UnsupportedOperationException(s"Cannot terminate expression: $generator") } + + def unableToCreateDatabaseAsFailedToCreateDirectoryError( + dbDefinition: CatalogDatabase, e: IOException): Throwable = { + new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + + s"to create its directory ${dbDefinition.locationUri}", e) + } + + def unableToDropDatabaseAsFailedToDeleteDirectoryError( + dbDefinition: CatalogDatabase, e: IOException): Throwable = { + new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + + s"to delete its directory ${dbDefinition.locationUri}", e) + } + + def unableToCreateTableAsFailedToCreateDirectoryError( + table: String, defaultTableLocation: Path, e: IOException): Throwable = { + new SparkException(s"Unable to create table $table as failed " + + s"to create its directory $defaultTableLocation", e) + } + + def unableToDeletePartitionPathError(partitionPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to delete partition path $partitionPath", e) + } + + def unableToDropTableAsFailedToDeleteDirectoryError( + table: String, dir: Path, e: IOException): Throwable = { + new SparkException(s"Unable to drop table $table as failed " + + s"to delete its directory $dir", e) + } + + def unableToRenameTableAsFailedToRenameDirectoryError( + oldName: String, newName: String, oldDir: Path, e: IOException): Throwable = { + new SparkException(s"Unable to rename table $oldName to $newName as failed " + + s"to rename its directory $oldDir", e) + } + + def unableToCreatePartitionPathError(partitionPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to create partition path $partitionPath", e) + } + + def unableToRenamePartitionPathError(oldPartPath: Path, e: IOException): Throwable = { + new SparkException(s"Unable to rename partition path $oldPartPath", e) + } + + def methodNotImplementedError(methodName: String): Throwable = { + new UnsupportedOperationException(s"$methodName is not implemented") + } + + def tableStatsNotSpecifiedError(): Throwable = { + new IllegalStateException("table stats must be specified.") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e41d3de642..bf5dbb8200 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2150,9 +2150,6 @@ class Analyzer(override val catalogManager: CatalogManager) lead.copy(ignoreNulls = ignoreNulls) case lag: Lag => lag.copy(ignoreNulls = ignoreNulls) - case _ => - throw QueryCompilationErrors.functionWithUnsupportedSyntaxError( - owf.prettyName, "IGNORE NULLS") } } else { owf @@ -2403,16 +2400,22 @@ class Analyzer(override val catalogManager: CatalogManager) // to push down this ordering expression and can reference the original aggregate // expression instead. val needsPushDown = ArrayBuffer.empty[NamedExpression] - val evaluatedOrderings = resolvedAliasedOrdering.zip(unresolvedSortOrders).map { - case (evaluated, order) => + val orderToAlias = unresolvedSortOrders.zip(aliasedOrdering) + val evaluatedOrderings = resolvedAliasedOrdering.zip(orderToAlias).map { + case (evaluated, (order, aliasOrder)) => val index = originalAggExprs.indexWhere { case Alias(child, _) => child semanticEquals evaluated.child case other => other semanticEquals evaluated.child } if (index == -1) { - needsPushDown += evaluated - order.copy(child = evaluated.toAttribute) + if (CharVarcharUtils.getRawType(evaluated.metadata).nonEmpty) { + needsPushDown += aliasOrder + order.copy(child = aliasOrder) + } else { + needsPushDown += evaluated + order.copy(child = evaluated.toAttribute) + } } else { order.copy(child = originalAggExprs(index).toAttribute) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala index 52e69480dc..47263791dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala @@ -67,7 +67,7 @@ case class UnresolvedView( case class UnresolvedTableOrView( multipartIdentifier: Seq[String], commandName: String, - allowTempView: Boolean = true) extends LeafNode { + allowTempView: Boolean) extends LeafNode { override lazy val resolved: Boolean = false override def output: Seq[Attribute] = Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala index 00445a1614..9d6e0a6d6c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogUtils.scala @@ -161,6 +161,10 @@ object ExternalCatalogUtils { } } + private def isNullPartitionValue(value: String): Boolean = { + value == null || value == DEFAULT_PARTITION_NAME + } + /** * Returns true if `spec1` is a partial partition spec w.r.t. `spec2`, e.g. PARTITION (a=1) is a * partial partition spec w.r.t. PARTITION (a=1,b=2). @@ -169,9 +173,15 @@ object ExternalCatalogUtils { spec1: TablePartitionSpec, spec2: TablePartitionSpec): Boolean = { spec1.forall { + case (partitionColumn, value) if isNullPartitionValue(value) => + isNullPartitionValue(spec2(partitionColumn)) case (partitionColumn, value) => spec2(partitionColumn) == value } } + + def convertNullPartitionValues(spec: TablePartitionSpec): TablePartitionSpec = { + spec.mapValues(v => if (v == null) DEFAULT_PARTITION_NAME else v).toMap + } } object CatalogUtils { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala index 6095ac0bc9..c7bd2a4cd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/GlobalTempViewManager.scala @@ -21,10 +21,10 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.errors.QueryCompilationErrors /** @@ -92,8 +92,7 @@ class GlobalTempViewManager(val database: String) { def rename(oldName: String, newName: String): Boolean = synchronized { if (viewDefinitions.contains(oldName)) { if (viewDefinitions.contains(newName)) { - throw new AnalysisException( - s"rename temporary view from '$oldName' to '$newName': destination view already exists") + throw QueryCompilationErrors.renameTempViewToExistingViewError(oldName, newName) } val viewDefinition = viewDefinitions(oldName) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala index 31644a5ae4..0d16f46d04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InMemoryCatalog.scala @@ -24,13 +24,13 @@ import scala.collection.mutable import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.sql.AnalysisException +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils._ import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.StructType /** @@ -112,8 +112,8 @@ class InMemoryCatalog( fs.mkdirs(location) } catch { case e: IOException => - throw new SparkException(s"Unable to create database ${dbDefinition.name} as failed " + - s"to create its directory ${dbDefinition.locationUri}", e) + throw QueryExecutionErrors.unableToCreateDatabaseAsFailedToCreateDirectoryError( + dbDefinition, e) } catalog.put(dbDefinition.name, new DatabaseDesc(dbDefinition)) } @@ -127,10 +127,10 @@ class InMemoryCatalog( if (!cascade) { // If cascade is false, make sure the database is empty. if (catalog(db).tables.nonEmpty) { - throw new AnalysisException(s"Database $db is not empty. One or more tables exist.") + throw QueryCompilationErrors.databaseNotEmptyError(db, "tables") } if (catalog(db).functions.nonEmpty) { - throw new AnalysisException(s"Database '$db' is not empty. One or more functions exist.") + throw QueryCompilationErrors.databaseNotEmptyError(db, "functions") } } // Remove the database. @@ -141,8 +141,8 @@ class InMemoryCatalog( fs.delete(location, true) } catch { case e: IOException => - throw new SparkException(s"Unable to drop database ${dbDefinition.name} as failed " + - s"to delete its directory ${dbDefinition.locationUri}", e) + throw QueryExecutionErrors.unableToDropDatabaseAsFailedToDeleteDirectoryError( + dbDefinition, e) } catalog.remove(db) } else { @@ -209,8 +209,8 @@ class InMemoryCatalog( fs.mkdirs(defaultTableLocation) } catch { case e: IOException => - throw new SparkException(s"Unable to create table $table as failed " + - s"to create its directory $defaultTableLocation", e) + throw QueryExecutionErrors.unableToCreateTableAsFailedToCreateDirectoryError( + table, defaultTableLocation, e) } tableDefinition.withNewStorage(locationUri = Some(defaultTableLocation.toUri)) } else { @@ -239,7 +239,7 @@ class InMemoryCatalog( fs.delete(partitionPath, true) } catch { case e: IOException => - throw new SparkException(s"Unable to delete partition path $partitionPath", e) + throw QueryExecutionErrors.unableToDeletePartitionPathError(partitionPath, e) } } assert(tableMeta.storage.locationUri.isDefined, @@ -252,8 +252,8 @@ class InMemoryCatalog( fs.delete(dir, true) } catch { case e: IOException => - throw new SparkException(s"Unable to drop table $table as failed " + - s"to delete its directory $dir", e) + throw QueryExecutionErrors.unableToDropTableAsFailedToDeleteDirectoryError( + table, dir, e) } } catalog(db).tables.remove(table) @@ -284,8 +284,8 @@ class InMemoryCatalog( fs.rename(oldDir, newDir) } catch { case e: IOException => - throw new SparkException(s"Unable to rename table $oldName to $newName as failed " + - s"to rename its directory $oldDir", e) + throw QueryExecutionErrors.unableToRenameTableAsFailedToRenameDirectoryError( + oldName, newName, oldDir, e) } oldDesc.table = oldDesc.table.withNewStorage(locationUri = Some(newDir.toUri)) } @@ -358,7 +358,7 @@ class InMemoryCatalog( loadPath: String, isOverwrite: Boolean, isSrcLocal: Boolean): Unit = { - throw new UnsupportedOperationException("loadTable is not implemented") + throw QueryExecutionErrors.methodNotImplementedError("loadTable") } override def loadPartition( @@ -369,7 +369,7 @@ class InMemoryCatalog( isOverwrite: Boolean, inheritTableSpecs: Boolean, isSrcLocal: Boolean): Unit = { - throw new UnsupportedOperationException("loadPartition is not implemented.") + throw QueryExecutionErrors.methodNotImplementedError("loadPartition") } override def loadDynamicPartitions( @@ -379,7 +379,7 @@ class InMemoryCatalog( partition: TablePartitionSpec, replace: Boolean, numDP: Int): Unit = { - throw new UnsupportedOperationException("loadDynamicPartitions is not implemented.") + throw QueryExecutionErrors.methodNotImplementedError("loadDynamicPartitions") } // -------------------------------------------------------------------------- @@ -416,7 +416,7 @@ class InMemoryCatalog( } } catch { case e: IOException => - throw new SparkException(s"Unable to create partition path $partitionPath", e) + throw QueryExecutionErrors.unableToCreatePartitionPathError(partitionPath, e) } existingParts.put( @@ -457,7 +457,7 @@ class InMemoryCatalog( fs.delete(partitionPath, true) } catch { case e: IOException => - throw new SparkException(s"Unable to delete partition path $partitionPath", e) + throw QueryExecutionErrors.unableToDeletePartitionPathError(partitionPath, e) } } existingParts.remove(p) @@ -490,7 +490,7 @@ class InMemoryCatalog( fs.rename(oldPartPath, newPartPath) } catch { case e: IOException => - throw new SparkException(s"Unable to rename partition path $oldPartPath", e) + throw QueryExecutionErrors.unableToRenamePartitionPathError(oldPartPath, e) } oldPartition.copy( spec = newSpec, @@ -541,7 +541,12 @@ class InMemoryCatalog( listPartitions(db, table, partialSpec).map { partition => partitionColumnNames.map { name => - escapePathName(name) + "=" + escapePathName(partition.spec(name)) + val partValue = if (partition.spec(name) == null) { + DEFAULT_PARTITION_NAME + } else { + escapePathName(partition.spec(name)) + } + escapePathName(name) + "=" + partValue }.mkString("/") }.sorted } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 9b542d6bd9..0428d12b7c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils} import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.types.StructType @@ -120,8 +121,7 @@ class SessionCatalog( */ private def validateName(name: String): Unit = { if (!validNameFormat.pattern.matcher(name).matches()) { - throw new AnalysisException(s"`$name` is not a valid name for tables/databases. " + - "Valid names only contain alphabet characters, numbers and _.") + throw QueryCompilationErrors.invalidNameForTableOrDatabaseError(name) } } @@ -216,9 +216,8 @@ class SessionCatalog( def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean): Unit = { val dbName = formatDatabaseName(dbDefinition.name) if (dbName == globalTempViewManager.database) { - throw new AnalysisException( - s"${globalTempViewManager.database} is a system preserved database, " + - "you cannot create a database with this name.") + throw QueryCompilationErrors.cannotCreateDatabaseWithSameNameAsPreservedDatabaseError( + globalTempViewManager.database) } validateName(dbName) externalCatalog.createDatabase( @@ -238,7 +237,7 @@ class SessionCatalog( def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = formatDatabaseName(db) if (dbName == DEFAULT_DATABASE) { - throw new AnalysisException(s"Can not drop default database") + throw QueryCompilationErrors.cannotDropDefaultDatabaseError } if (cascade && databaseExists(dbName)) { listTables(dbName).foreach { t => @@ -279,11 +278,8 @@ class SessionCatalog( def setCurrentDatabase(db: String): Unit = { val dbName = formatDatabaseName(db) if (dbName == globalTempViewManager.database) { - throw new AnalysisException( - s"${globalTempViewManager.database} is a system preserved database, " + - "you cannot use it as current database. To access global temporary views, you should " + - "use qualified name with the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM " + - s"${globalTempViewManager.database}.viewName.") + throw QueryCompilationErrors.cannotUsePreservedDatabaseAsCurrentDatabaseError( + globalTempViewManager.database) } requireDbExists(dbName) synchronized { currentDb = dbName } @@ -320,7 +316,7 @@ class SessionCatalog( validateLocation: Boolean = true): Unit = { val isExternal = tableDefinition.tableType == CatalogTableType.EXTERNAL if (isExternal && tableDefinition.storage.locationUri.isEmpty) { - throw new AnalysisException(s"CREATE EXTERNAL TABLE must be accompanied by LOCATION") + throw QueryCompilationErrors.createExternalTableWithoutLocationError } val db = formatDatabaseName(tableDefinition.identifier.database.getOrElse(getCurrentDatabase)) @@ -359,8 +355,8 @@ class SessionCatalog( val fs = tableLocation.getFileSystem(hadoopConf) if (fs.exists(tableLocation) && fs.listStatus(tableLocation).nonEmpty) { - throw new AnalysisException(s"Can not create the managed table('${table.identifier}')" + - s". The associated location('${tableLocation.toString}') already exists.") + throw QueryCompilationErrors.cannotOperateManagedTableWithExistingLocationError( + "create", table.identifier, tableLocation) } } } @@ -428,11 +424,7 @@ class SessionCatalog( val nonExistentColumnNames = oldDataSchema.map(_.name).filterNot(columnNameResolved(newDataSchema, _)) if (nonExistentColumnNames.nonEmpty) { - throw new AnalysisException( - s""" - |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are - |not present in the new schema. We don't support dropping columns yet. - """.stripMargin) + throw QueryCompilationErrors.dropNonExistentColumnsNotSupportedError(nonExistentColumnNames) } externalCatalog.alterTableDataSchema(db, table, newDataSchema) @@ -508,10 +500,8 @@ class SessionCatalog( if (dbs.distinct.size != 1) { val tables = names.map(name => formatTableName(name.table)) val qualifiedTableNames = dbs.zip(tables).map { case (d, t) => QualifiedTableName(d, t)} - throw new AnalysisException( - s"Only the tables/views belong to the same database can be retrieved. Querying " + - s"tables/views are $qualifiedTableNames" - ) + throw QueryCompilationErrors.cannotRetrieveTableOrViewNotInSameDatabaseError( + qualifiedTableNames) } val db = formatDatabaseName(dbs.head) requireDbExists(db) @@ -722,8 +712,7 @@ class SessionCatalog( val db = formatDatabaseName(oldName.database.getOrElse(currentDb)) newName.database.map(formatDatabaseName).foreach { newDb => if (db != newDb) { - throw new AnalysisException( - s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'") + throw QueryCompilationErrors.renameTableSourceAndDestinationMismatchError(db, newDb) } } @@ -741,13 +730,12 @@ class SessionCatalog( externalCatalog.renameTable(db, oldTableName, newTableName) } else { if (newName.database.isDefined) { - throw new AnalysisException( - s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': cannot specify database " + - s"name '${newName.database.get}' in the destination table") + throw QueryCompilationErrors.cannotRenameTempViewWithDatabaseSpecifiedError( + oldName, newName) } if (tempViews.contains(newTableName)) { - throw new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " + - "destination table already exists") + throw QueryCompilationErrors.cannotRenameTempViewToExistingTableError( + oldName, newName) } val table = tempViews(oldTableName) tempViews.remove(oldTableName) @@ -1190,10 +1178,10 @@ class SessionCatalog( */ private def requireNonEmptyValueInPartitionSpec(specs: Seq[TablePartitionSpec]): Unit = { specs.foreach { s => - if (s.values.exists(_.isEmpty)) { + if (s.values.exists(v => v != null && v.isEmpty)) { val spec = s.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") - throw new AnalysisException( - s"Partition spec is invalid. The spec ($spec) contains an empty partition column value") + throw QueryCompilationErrors.invalidPartitionSpecError( + s"The spec ($spec) contains an empty partition column value") } } } @@ -1223,10 +1211,10 @@ class SessionCatalog( val defined = table.partitionColumnNames specs.foreach { s => if (!s.keys.forall(defined.contains)) { - throw new AnalysisException( - s"Partition spec is invalid. The spec (${s.keys.mkString(", ")}) must be contained " + - s"within the partition spec (${table.partitionColumnNames.mkString(", ")}) defined " + - s"in table '${table.identifier}'") + throw QueryCompilationErrors.invalidPartitionSpecError( + s"The spec (${s.keys.mkString(", ")}) must be contained " + + s"within the partition spec (${table.partitionColumnNames.mkString(", ")}) defined " + + s"in table '${table.identifier}'") } } } @@ -1382,8 +1370,8 @@ class SessionCatalog( // Check input argument size if (e.inputTypes.size != input.size) { - throw new AnalysisException(s"Invalid number of arguments for function $name. " + - s"Expected: ${e.inputTypes.size}; Found: ${input.size}") + throw QueryCompilationErrors.invalidFunctionArgumentsError( + name, e.inputTypes.size.toString, input.size) } e } else { @@ -1409,15 +1397,14 @@ class SessionCatalog( functionBuilder: Option[FunctionBuilder] = None): Unit = { val func = funcDefinition.identifier if (functionRegistry.functionExists(func) && !overrideIfExists) { - throw new AnalysisException(s"Function $func already exists") + throw QueryCompilationErrors.functionAlreadyExistsError(func) } val info = new ExpressionInfo(funcDefinition.className, func.database.orNull, func.funcName) val builder = functionBuilder.getOrElse { val className = funcDefinition.className if (!Utils.classIsLoadable(className)) { - throw new AnalysisException(s"Can not load class '$className' when registering " + - s"the function '$func', please make sure it is on the classpath") + throw QueryCompilationErrors.cannotLoadClassWhenRegisteringFunctionError(className, func) } makeFunctionBuilder(func.unquotedString, className) } @@ -1522,7 +1509,6 @@ class SessionCatalog( def lookupFunction( name: FunctionIdentifier, children: Seq[Expression]): Expression = synchronized { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ // Note: the implementation of this function is a little bit convoluted. // We probably shouldn't use a single FunctionRegistry to register all three kinds of functions // (built-in, temp, and external). @@ -1545,9 +1531,7 @@ class SessionCatalog( case Seq() => getCurrentDatabase case Seq(_, db) => db case Seq(catalog, namespace @ _*) => - throw new AnalysisException( - s"V2 catalog does not support functions yet. " + - s"catalog: ${catalog}, namespace: '${namespace.quoted}'") + throw QueryCompilationErrors.v2CatalogNotSupportFunctionError(catalog, namespace) } // If the name itself is not qualified, add the current database to it. @@ -1568,7 +1552,6 @@ class SessionCatalog( externalCatalog.getFunction(database, name.funcName) } catch { case _: AnalysisException => failFunctionLookup(name) - case _: NoSuchPermanentFunctionException => failFunctionLookup(name) } loadFunctionResources(catalogFunction.resources) // Please note that qualifiedName is provided by the user. However, @@ -1685,8 +1668,8 @@ class SessionCatalog( val newTableLocation = new Path(new Path(databaseLocation), formatTableName(newName.table)) val fs = newTableLocation.getFileSystem(hadoopConf) if (fs.exists(newTableLocation)) { - throw new AnalysisException(s"Can not rename the managed table('$oldName')" + - s". The associated location('$newTableLocation') already exists.") + throw QueryCompilationErrors.cannotOperateManagedTableWithExistingLocationError( + "rename", oldName, newTableLocation) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala index 67bf2d06c9..7ebe3d8c5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/functionResources.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.catalog import java.util.Locale -import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.errors.QueryCompilationErrors /** A trait that represents the type of a resourced needed by a function. */ abstract class FunctionResourceType(val resourceType: String) @@ -40,7 +40,7 @@ object FunctionResourceType { case "file" => FileResource case "archive" => ArchiveResource case other => - throw new AnalysisException(s"Resource Type '$resourceType' is not supported.") + throw QueryCompilationErrors.resourceTypeNotSupportedError(resourceType) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index d25b1fe46d..c715dacf69 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -29,7 +29,6 @@ import org.json4s.JsonAST.{JArray, JString} import org.json4s.jackson.JsonMethods._ import org.apache.spark.internal.Logging -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, SQLConfHelper, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Cast, ExprId, Literal} @@ -37,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -145,7 +145,7 @@ case class CatalogTablePartition( /** Return the partition location, assuming it is specified. */ def location: URI = storage.locationUri.getOrElse { val specString = spec.map { case (k, v) => s"$k=$v" }.mkString(", ") - throw new AnalysisException(s"Partition [$specString] did not specify locationUri") + throw QueryCompilationErrors.partitionNotSpecifyLocationUriError(specString) } /** @@ -182,9 +182,8 @@ case class BucketSpec( sortColumnNames: Seq[String]) extends SQLConfHelper { if (numBuckets <= 0 || numBuckets > conf.bucketingMaxBuckets) { - throw new AnalysisException( - s"Number of buckets should be greater than 0 but less than or equal to " + - s"bucketing.maxBuckets (`${conf.bucketingMaxBuckets}`). Got `$numBuckets`") + throw QueryCompilationErrors.invalidBucketNumberError( + conf.bucketingMaxBuckets, numBuckets) } override def toString: String = { @@ -274,12 +273,12 @@ case class CatalogTable( /** Return the database this table was specified to belong to, assuming it exists. */ def database: String = identifier.database.getOrElse { - throw new AnalysisException(s"table $identifier did not specify database") + throw QueryCompilationErrors.tableNotSpecifyDatabaseError(identifier) } /** Return the table location, assuming it is specified. */ def location: URI = storage.locationUri.getOrElse { - throw new AnalysisException(s"table $identifier did not specify locationUri") + throw QueryCompilationErrors.tableNotSpecifyLocationUriError(identifier) } /** Return the fully qualified name of this table, assuming the database was specified. */ @@ -295,8 +294,7 @@ case class CatalogTable( (0 until numParts).map { index => properties.getOrElse( s"$VIEW_CATALOG_AND_NAMESPACE_PART_PREFIX$index", - throw new AnalysisException("Corrupted table name context in catalog: " + - s"$numParts parts expected, but part $index is missing.") + throw QueryCompilationErrors.corruptedTableNameContextInCatalogError(numParts, index) ) } } else if (properties.contains(VIEW_DEFAULT_DATABASE)) { @@ -318,8 +316,7 @@ case class CatalogTable( yield (key.substring(CatalogTable.VIEW_SQL_CONFIG_PREFIX.length), value) } catch { case e: Exception => - throw new AnalysisException( - "Corrupted view SQL configs in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewSQLConfigsInCatalogError(e) } } @@ -334,8 +331,7 @@ case class CatalogTable( index <- 0 until numCols.toInt } yield properties.getOrElse( s"$VIEW_QUERY_OUTPUT_COLUMN_NAME_PREFIX$index", - throw new AnalysisException("Corrupted view query output column names in catalog: " + - s"$numCols parts expected, but part $index is missing.") + throw QueryCompilationErrors.corruptedViewQueryOutputColumnsInCatalogError(numCols, index) ) } @@ -352,8 +348,7 @@ case class CatalogTable( }.getOrElse(Seq.empty) } catch { case e: Exception => - throw new AnalysisException( - "corrupted view referred temp view names in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewReferredTempViewInCatalogError(e) } } @@ -368,8 +363,7 @@ case class CatalogTable( }.getOrElse(Seq.empty) } catch { case e: Exception => - throw new AnalysisException( - "corrupted view referred temp functions names in catalog", cause = Some(e)) + throw QueryCompilationErrors.corruptedViewReferredTempFunctionsInCatalogError(e) } } @@ -497,14 +491,13 @@ object CatalogTable { None } else { val numParts = props.get(s"$key.numParts") - val errorMessage = s"Cannot read table property '$key' as it's corrupted." if (numParts.isEmpty) { - throw new AnalysisException(errorMessage) + throw QueryCompilationErrors.cannotReadCorruptedTablePropertyError(key) } else { val parts = (0 until numParts.get.toInt).map { index => props.getOrElse(s"$key.part.$index", { - throw new AnalysisException( - s"$errorMessage Missing part $index, ${numParts.get} parts are expected.") + throw QueryCompilationErrors.cannotReadCorruptedTablePropertyError( + key, s"Missing part $index, $numParts parts are expected.") }) } Some(parts.mkString) @@ -517,6 +510,30 @@ object CatalogTable { propKey == originalKey || propKey == s"$originalKey.numParts" || propKey.startsWith(s"$originalKey.part.") } + + def normalize(table: CatalogTable): CatalogTable = { + val nondeterministicProps = Set( + "CreateTime", + "transient_lastDdlTime", + "grantTime", + "lastUpdateTime", + "last_modified_by", + "last_modified_time", + "Owner:", + // The following are hive specific schema parameters which we do not need to match exactly. + "totalNumberFiles", + "maxFileSize", + "minFileSize" + ) + + table.copy( + createTime = 0L, + lastAccessTime = 0L, + properties = table.properties.filterKeys(!nondeterministicProps.contains(_)).toMap, + stats = None, + ignoredProperties = Map.empty + ) + } } /** @@ -657,8 +674,8 @@ object CatalogColumnStat extends Logging { // This version of Spark does not use min/max for binary/string types so we ignore it. case BinaryType | StringType => null case _ => - throw new AnalysisException("Column statistics deserialization is not supported for " + - s"column $name of data type: $dataType.") + throw QueryCompilationErrors.columnStatisticsDeserializationNotSupportedError( + name, dataType) } } @@ -674,8 +691,8 @@ object CatalogColumnStat extends Logging { case _: DecimalType => v.asInstanceOf[Decimal].toJavaBigDecimal // This version of Spark does not use min/max for binary/string types so we ignore it. case _ => - throw new AnalysisException("Column statistics serialization is not supported for " + - s"column $colName of data type: $dataType.") + throw QueryCompilationErrors.columnStatisticsSerializationNotSupportedError( + colName, dataType) } externalValue.toString } @@ -788,10 +805,7 @@ case class HiveTableRelation( def isPartitioned: Boolean = partitionCols.nonEmpty override def doCanonicalize(): HiveTableRelation = copy( - tableMeta = tableMeta.copy( - storage = CatalogStorageFormat.empty, - createTime = -1 - ), + tableMeta = CatalogTable.normalize(tableMeta), dataCols = dataCols.zipWithIndex.map { case (attr, index) => attr.withExprId(ExprId(index)) }, @@ -805,7 +819,7 @@ case class HiveTableRelation( tableMeta.stats.map(_.toPlanStats(output, conf.cboEnabled || conf.planStatsEnabled)) .orElse(tableStats) .getOrElse { - throw new IllegalStateException("table stats must be specified.") + throw QueryExecutionErrors.tableStatsNotSpecifiedError } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index 99f80e9078..c9a9ac3855 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1162,7 +1162,12 @@ case class LastDay(startDate: Expression) */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(start_date, day_of_week) - Returns the first date which is later than `start_date` and named as indicated.", + usage = + """_FUNC_(start_date, day_of_week) - Returns the first date which is later than `start_date` and named as indicated. + The function returns NULL if at least one of the input parameters is NULL. + When both of the input parameters are not NULL and day_of_week is an invalid input, + the function throws IllegalArgumentException if `spark.sql.ansi.enabled` is set to true, otherwise NULL. + """, examples = """ Examples: > SELECT _FUNC_('2015-01-14', 'TU'); @@ -1171,52 +1176,73 @@ case class LastDay(startDate: Expression) group = "datetime_funcs", since = "1.5.0") // scalastyle:on line.size.limit -case class NextDay(startDate: Expression, dayOfWeek: Expression) +case class NextDay( + startDate: Expression, + dayOfWeek: Expression, + failOnError: Boolean = SQLConf.get.ansiEnabled) extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant { override def left: Expression = startDate override def right: Expression = dayOfWeek + def this(left: Expression, right: Expression) = this(left, right, SQLConf.get.ansiEnabled) + override def inputTypes: Seq[AbstractDataType] = Seq(DateType, StringType) override def dataType: DataType = DateType override def nullable: Boolean = true override def nullSafeEval(start: Any, dayOfW: Any): Any = { - val dow = DateTimeUtils.getDayOfWeekFromString(dayOfW.asInstanceOf[UTF8String]) - if (dow == -1) { - null - } else { + try { + val dow = DateTimeUtils.getDayOfWeekFromString(dayOfW.asInstanceOf[UTF8String]) val sd = start.asInstanceOf[Int] DateTimeUtils.getNextDateForDayOfWeek(sd, dow) + } catch { + case _: IllegalArgumentException if !failOnError => null + } + } + + private def dateTimeUtilClass: String = DateTimeUtils.getClass.getName.stripSuffix("$") + + private def nextDayGenCode( + ev: ExprCode, + dayOfWeekTerm: String, + sd: String, + dowS: String): String = { + if (failOnError) { + s""" + |int $dayOfWeekTerm = $dateTimeUtilClass.getDayOfWeekFromString($dowS); + |${ev.value} = $dateTimeUtilClass.getNextDateForDayOfWeek($sd, $dayOfWeekTerm); + |""".stripMargin + } else { + s""" + |try { + | int $dayOfWeekTerm = $dateTimeUtilClass.getDayOfWeekFromString($dowS); + | ${ev.value} = $dateTimeUtilClass.getNextDateForDayOfWeek($sd, $dayOfWeekTerm); + |} catch (IllegalArgumentException e) { + | ${ev.isNull} = true; + |} + |""".stripMargin } } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (sd, dowS) => { - val dateTimeUtilClass = DateTimeUtils.getClass.getName.stripSuffix("$") val dayOfWeekTerm = ctx.freshName("dayOfWeek") if (dayOfWeek.foldable) { val input = dayOfWeek.eval().asInstanceOf[UTF8String] - if ((input eq null) || DateTimeUtils.getDayOfWeekFromString(input) == -1) { - s""" - |${ev.isNull} = true; - """.stripMargin + if (input eq null) { + s"""${ev.isNull} = true;""" } else { - val dayOfWeekValue = DateTimeUtils.getDayOfWeekFromString(input) - s""" - |${ev.value} = $dateTimeUtilClass.getNextDateForDayOfWeek($sd, $dayOfWeekValue); - """.stripMargin + try { + val dayOfWeekValue = DateTimeUtils.getDayOfWeekFromString(input) + s"${ev.value} = $dateTimeUtilClass.getNextDateForDayOfWeek($sd, $dayOfWeekValue);" + } catch { + case _: IllegalArgumentException => nextDayGenCode(ev, dayOfWeekTerm, sd, dowS) + } } } else { - s""" - |int $dayOfWeekTerm = $dateTimeUtilClass.getDayOfWeekFromString($dowS); - |if ($dayOfWeekTerm == -1) { - | ${ev.isNull} = true; - |} else { - | ${ev.value} = $dateTimeUtilClass.getNextDateForDayOfWeek($sd, $dayOfWeekTerm); - |} - """.stripMargin + nextDayGenCode(ev, dayOfWeekTerm, sd, dowS) } }) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index 9303df75af..f391b3128c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -954,7 +954,7 @@ case class MapObjects private( } else { doCodeGenForScala213 } - case Some(cls) if classOf[Seq[_]].isAssignableFrom(cls) || + case Some(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) || classOf[scala.collection.Set[_]].isAssignableFrom(cls) => // Scala sequence or set val getBuilder = s"${cls.getName}$$.MODULE$$.newBuilder()" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index dae954a579..011371a513 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -181,7 +181,7 @@ case class Like(left: Expression, right: Expression, escapeChar: Char) } } -abstract class MultiLikeBase +sealed abstract class MultiLikeBase extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { protected def patterns: Seq[UTF8String] @@ -220,7 +220,7 @@ abstract class MultiLikeBase /** * Optimized version of LIKE ALL, when all pattern values are literal. */ -abstract class LikeAllBase extends MultiLikeBase { +sealed abstract class LikeAllBase extends MultiLikeBase { override def matches(exprValue: String): Any = { if (cache.forall(matchFunc(_, exprValue))) { @@ -276,7 +276,7 @@ case class NotLikeAll(child: Expression, patterns: Seq[UTF8String]) extends Like /** * Optimized version of LIKE ANY, when all pattern values are literal. */ -abstract class LikeAnyBase extends MultiLikeBase { +sealed abstract class LikeAnyBase extends MultiLikeBase { override def matches(exprValue: String): Any = { if (cache.exists(matchFunc(_, exprValue))) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index b167499620..1934a9b190 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -369,7 +369,7 @@ trait OffsetWindowFunction extends WindowFunction { * within the partition. For instance: a FrameLessOffsetWindowFunction for value x with offset -2, * will get the value of x 2 rows back from the current row in the partition. */ -abstract class FrameLessOffsetWindowFunction +sealed abstract class FrameLessOffsetWindowFunction extends OffsetWindowFunction with Unevaluable with ImplicitCastInputTypes { override def children: Seq[Expression] = Seq(input, offset, default) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 11b675e758..c41686da79 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -349,14 +349,11 @@ object JoinReorderDP extends PredicateHelper with Logging { } def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - if (other.planCost.card == 0 || other.planCost.size == 0) { - false - } else { - val relativeRows = BigDecimal(this.planCost.card) / BigDecimal(other.planCost.card) - val relativeSize = BigDecimal(this.planCost.size) / BigDecimal(other.planCost.size) - relativeRows * conf.joinReorderCardWeight + - relativeSize * (1 - conf.joinReorderCardWeight) < 1 - } + val thisCost = BigDecimal(this.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(this.planCost.size) * (1 - conf.joinReorderCardWeight) + val otherCost = BigDecimal(other.planCost.card) * conf.joinReorderCardWeight + + BigDecimal(other.planCost.size) * (1 - conf.joinReorderCardWeight) + thisCost < otherCost } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index f61fad7c3e..28dfc0958a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -88,6 +88,7 @@ abstract class Optimizer(catalogManager: CatalogManager) EliminateLimits, CombineUnions, // Constant folding and strength reduction + OptimizeRepartition, TransposeWindow, NullPropagation, ConstantPropagation, @@ -818,6 +819,19 @@ object CollapseRepartition extends Rule[LogicalPlan] { } } +/** + * Replace RepartitionByExpression numPartitions to 1 if all partition expressions are foldable + * and user not specify. + */ +object OptimizeRepartition extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan.transform { + case r @ RepartitionByExpression(partitionExpressions, _, numPartitions) + if partitionExpressions.nonEmpty && partitionExpressions.forall(_.foldable) && + numPartitions.isEmpty => + r.copy(optNumPartitions = Some(1)) + } +} + /** * Replaces first(col) to nth_value(col, 1) for better performance. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala index 1ea85085bc..1225f1f318 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicate.scala @@ -39,9 +39,7 @@ import org.apache.spark.sql.types.BooleanType * - CASE WHEN cond THEN trueVal ELSE null END => AND(cond, trueVal) * - CASE WHEN cond THEN trueVal ELSE true END => OR(NOT(cond), trueVal) * - CASE WHEN cond THEN false ELSE elseVal END => AND(NOT(cond), elseVal) - * - CASE WHEN cond THEN false END => false * - CASE WHEN cond THEN true ELSE elseVal END => OR(cond, elseVal) - * - CASE WHEN cond THEN true END => cond */ object SimplifyConditionalsInPredicate extends Rule[LogicalPlan] { @@ -64,12 +62,8 @@ object SimplifyConditionalsInPredicate extends Rule[LogicalPlan] { And(cond, trueValue) case CaseWhen(Seq((cond, trueValue)), Some(TrueLiteral)) => Or(Not(cond), trueValue) - case CaseWhen(Seq((_, FalseLiteral)), Some(FalseLiteral) | None) => - FalseLiteral case CaseWhen(Seq((cond, FalseLiteral)), Some(elseValue)) => And(Not(cond), elseValue) - case CaseWhen(Seq((cond, TrueLiteral)), Some(FalseLiteral) | None) => - cond case CaseWhen(Seq((cond, TrueLiteral)), Some(elseValue)) => Or(cond, elseValue) case e if e.dataType == BooleanType => e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 819bffeafb..a40456da82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -21,7 +21,7 @@ import scala.collection.immutable.HashSet import scala.collection.mutable.{ArrayBuffer, Stack} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, _} +import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, MultiLikeBase, _} import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String /* * Optimization rules defined in this file should not affect the structure of the logical plan. @@ -634,36 +635,68 @@ object LikeSimplification extends Rule[LogicalPlan] { private val contains = "%([^_%]+)%".r private val equalTo = "([^_%]*)".r + private def simplifyLike( + input: Expression, pattern: String, escapeChar: Char = '\\'): Option[Expression] = { + if (pattern.contains(escapeChar)) { + // There are three different situations when pattern containing escapeChar: + // 1. pattern contains invalid escape sequence, e.g. 'm\aca' + // 2. pattern contains escaped wildcard character, e.g. 'ma\%ca' + // 3. pattern contains escaped escape character, e.g. 'ma\\ca' + // Although there are patterns can be optimized if we handle the escape first, we just + // skip this rule if pattern contains any escapeChar for simplicity. + None + } else { + pattern match { + case startsWith(prefix) => + Some(StartsWith(input, Literal(prefix))) + case endsWith(postfix) => + Some(EndsWith(input, Literal(postfix))) + // 'a%a' pattern is basically same with 'a%' && '%a'. + // However, the additional `Length` condition is required to prevent 'a' match 'a%a'. + case startsAndEndsWith(prefix, postfix) => + Some(And(GreaterThanOrEqual(Length(input), Literal(prefix.length + postfix.length)), + And(StartsWith(input, Literal(prefix)), EndsWith(input, Literal(postfix))))) + case contains(infix) => + Some(Contains(input, Literal(infix))) + case equalTo(str) => + Some(EqualTo(input, Literal(str))) + case _ => None + } + } + } + + private def simplifyMultiLike( + child: Expression, patterns: Seq[UTF8String], multi: MultiLikeBase): Expression = { + val (remainPatternMap, replacementMap) = + patterns.map { p => p -> simplifyLike(child, p.toString)}.partition(_._2.isEmpty) + val remainPatterns = remainPatternMap.map(_._1) + val replacements = replacementMap.map(_._2.get) + if (replacements.isEmpty) { + multi + } else { + multi match { + case l: LikeAll => And(replacements.reduceLeft(And), l.copy(patterns = remainPatterns)) + case l: NotLikeAll => + And(replacements.map(Not(_)).reduceLeft(And), l.copy(patterns = remainPatterns)) + case l: LikeAny => Or(replacements.reduceLeft(Or), l.copy(patterns = remainPatterns)) + case l: NotLikeAny => + Or(replacements.map(Not(_)).reduceLeft(Or), l.copy(patterns = remainPatterns)) + } + } + } + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case l @ Like(input, Literal(pattern, StringType), escapeChar) => if (pattern == null) { // If pattern is null, return null value directly, since "col like null" == null. Literal(null, BooleanType) } else { - pattern.toString match { - // There are three different situations when pattern containing escapeChar: - // 1. pattern contains invalid escape sequence, e.g. 'm\aca' - // 2. pattern contains escaped wildcard character, e.g. 'ma\%ca' - // 3. pattern contains escaped escape character, e.g. 'ma\\ca' - // Although there are patterns can be optimized if we handle the escape first, we just - // skip this rule if pattern contains any escapeChar for simplicity. - case p if p.contains(escapeChar) => l - case startsWith(prefix) => - StartsWith(input, Literal(prefix)) - case endsWith(postfix) => - EndsWith(input, Literal(postfix)) - // 'a%a' pattern is basically same with 'a%' && '%a'. - // However, the additional `Length` condition is required to prevent 'a' match 'a%a'. - case startsAndEndsWith(prefix, postfix) => - And(GreaterThanOrEqual(Length(input), Literal(prefix.length + postfix.length)), - And(StartsWith(input, Literal(prefix)), EndsWith(input, Literal(postfix)))) - case contains(infix) => - Contains(input, Literal(infix)) - case equalTo(str) => - EqualTo(input, Literal(str)) - case _ => l - } + simplifyLike(input, pattern.toString, escapeChar).getOrElse(l) } + case l @ LikeAll(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ NotLikeAll(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ LikeAny(child, patterns) => simplifyMultiLike(child, patterns, l) + case l @ NotLikeAny(child, patterns) => simplifyMultiLike(child, patterns, l) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 3ea86c6ea2..5016d1d331 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.trees.CurrentOrigin import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, IntervalUtils} import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.catalyst.util.IntervalUtils.IntervalUnit @@ -511,6 +512,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ protected def visitStringConstant(ctx: ConstantContext): String = withOrigin(ctx) { ctx match { + case _: NullLiteralContext => null case s: StringLiteralContext => createString(s) case o => o.getText } @@ -714,7 +716,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val withProject = if (aggregationClause == null && havingClause != null) { if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. - withHavingClause(havingClause, createProject()) + val predicate = expression(havingClause.booleanExpression) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, createProject()) } else { // According to SQL standard, HAVING without GROUP BY means global aggregate. withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter)) @@ -2167,7 +2173,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ private def createUnresolvedTable( ctx: MultipartIdentifierContext, - commandName: String): LogicalPlan = withOrigin(ctx) { + commandName: String): UnresolvedTable = withOrigin(ctx) { UnresolvedTable(visitMultipartIdentifier(ctx), commandName) } @@ -2178,10 +2184,20 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg ctx: MultipartIdentifierContext, commandName: String, allowTemp: Boolean = true, - relationTypeMismatchHint: Option[String] = None): LogicalPlan = withOrigin(ctx) { + relationTypeMismatchHint: Option[String] = None): UnresolvedView = withOrigin(ctx) { UnresolvedView(visitMultipartIdentifier(ctx), commandName, allowTemp, relationTypeMismatchHint) } + /** + * Create an [[UnresolvedTableOrView]] from a multi-part identifier context. + */ + private def createUnresolvedTableOrView( + ctx: MultipartIdentifierContext, + commandName: String, + allowTempView: Boolean = true): UnresolvedTableOrView = withOrigin(ctx) { + UnresolvedTableOrView(visitMultipartIdentifier(ctx), commandName, allowTempView) + } + /** * Create a [[CalendarInterval]] literal expression. Two syntaxes are supported: * - multiple unit value pairs, for instance: interval 2 months 2 days. @@ -3218,7 +3234,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { // DROP TABLE works with either a table or a temporary view. DropTable( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier()), "DROP TABLE"), + createUnresolvedTableOrView(ctx.multipartIdentifier(), "DROP TABLE"), ctx.EXISTS != null, ctx.PURGE != null) } @@ -3548,8 +3564,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitDescribeRelation(ctx: DescribeRelationContext): LogicalPlan = withOrigin(ctx) { val isExtended = ctx.EXTENDED != null || ctx.FORMATTED != null - val relation = UnresolvedTableOrView( - visitMultipartIdentifier(ctx.multipartIdentifier()), + val relation = createUnresolvedTableOrView( + ctx.multipartIdentifier(), "DESCRIBE TABLE") if (ctx.describeColName != null) { if (ctx.partitionSpec != null) { @@ -3606,11 +3622,12 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg ctx.identifier()) } - val tableName = visitMultipartIdentifier(ctx.multipartIdentifier()) if (ctx.ALL() != null) { checkPartitionSpec() AnalyzeColumn( - UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR ALL COLUMNS"), + createUnresolvedTableOrView( + ctx.multipartIdentifier(), + "ANALYZE TABLE ... FOR ALL COLUMNS"), None, allColumns = true) } else if (ctx.identifierSeq() == null) { @@ -3620,13 +3637,18 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg Map.empty[String, Option[String]] } AnalyzeTable( - UnresolvedTableOrView(tableName, "ANALYZE TABLE", allowTempView = false), + createUnresolvedTableOrView( + ctx.multipartIdentifier(), + "ANALYZE TABLE", + allowTempView = false), partitionSpec, noScan = ctx.identifier != null) } else { checkPartitionSpec() AnalyzeColumn( - UnresolvedTableOrView(tableName, "ANALYZE TABLE ... FOR COLUMNS ..."), + createUnresolvedTableOrView( + ctx.multipartIdentifier(), + "ANALYZE TABLE ... FOR COLUMNS ..."), Option(visitIdentifierSeq(ctx.identifierSeq())), allColumns = false) } @@ -3668,8 +3690,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitShowCreateTable(ctx: ShowCreateTableContext): LogicalPlan = withOrigin(ctx) { ShowCreateTable( - UnresolvedTableOrView( - visitMultipartIdentifier(ctx.multipartIdentifier()), + createUnresolvedTableOrView( + ctx.multipartIdentifier(), "SHOW CREATE TABLE", allowTempView = false), ctx.SERDE != null) @@ -3757,8 +3779,8 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg */ override def visitRefreshTable(ctx: RefreshTableContext): LogicalPlan = withOrigin(ctx) { RefreshTable( - UnresolvedTableOrView( - visitMultipartIdentifier(ctx.multipartIdentifier()), + createUnresolvedTableOrView( + ctx.multipartIdentifier(), "REFRESH TABLE")) } @@ -3773,16 +3795,18 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg * }}} */ override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) { - val nameParts = visitMultipartIdentifier(ctx.table) + val table = createUnresolvedTableOrView(ctx.table, "SHOW COLUMNS") val namespace = Option(ctx.ns).map(visitMultipartIdentifier) // Use namespace only if table name doesn't specify it. If namespace is already specified // in the table name, it's checked against the given namespace after table/view is resolved. - val tableName = if (namespace.isDefined && nameParts.length == 1) { - namespace.get ++ nameParts + val tableWithNamespace = if (namespace.isDefined && table.multipartIdentifier.length == 1) { + CurrentOrigin.withOrigin(table.origin) { + table.copy(multipartIdentifier = namespace.get ++ table.multipartIdentifier) + } } else { - nameParts + table } - ShowColumns(UnresolvedTableOrView(tableName, "SHOW COLUMNS"), namespace) + ShowColumns(tableWithNamespace, namespace) } /** @@ -3984,9 +4008,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg val isView = ctx.VIEW != null val relationStr = if (isView) "VIEW" else "TABLE" RenameTable( - UnresolvedTableOrView( - visitMultipartIdentifier(ctx.from), - s"ALTER $relationStr ... RENAME TO"), + createUnresolvedTableOrView(ctx.from, s"ALTER $relationStr ... RENAME TO"), visitMultipartIdentifier(ctx.to), isView) } @@ -4003,7 +4025,7 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg override def visitShowTblProperties( ctx: ShowTblPropertiesContext): LogicalPlan = withOrigin(ctx) { ShowTableProperties( - UnresolvedTableOrView(visitMultipartIdentifier(ctx.table), "SHOW TBLPROPERTIES"), + createUnresolvedTableOrView(ctx.table, "SHOW TBLPROPERTIES"), Option(ctx.key).map(visitTablePropertyKey)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala index ffd1f784e4..54b0141638 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AnalysisHelper.scala @@ -133,6 +133,15 @@ trait AnalysisHelper extends QueryPlan[LogicalPlan] { self: LogicalPlan => } } + override def transformUpWithNewOutput( + rule: PartialFunction[LogicalPlan, (LogicalPlan, Seq[(Attribute, Attribute)])], + skipCond: LogicalPlan => Boolean, + canGetOutput: LogicalPlan => Boolean): LogicalPlan = { + AnalysisHelper.allowInvokingTransformsInAnalyzer { + super.transformUpWithNewOutput(rule, skipCond, canGetOutput) + } + } + /** * Recursively transforms the expressions of a tree, skipping nodes that have already * been analyzed. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index ee7db7ae83..80883e1cd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -217,7 +217,7 @@ object Union { } /** - * Logical plan for unioning two plans, without a distinct. This is UNION ALL in SQL. + * Logical plan for unioning multiple plans, without a distinct. This is UNION ALL in SQL. * * @param byName Whether resolves columns in the children by column names. * @param allowMissingCol Allows missing columns in children query plans. If it is true, @@ -1047,16 +1047,7 @@ case class RepartitionByExpression( child: LogicalPlan, optNumPartitions: Option[Int]) extends RepartitionOperation { - val numPartitions = if (optNumPartitions.nonEmpty) { - optNumPartitions.get - } else { - if (partitionExpressions.forall(_.foldable)) { - 1 - } else { - SQLConf.get.numShufflePartitions - } - } - + val numPartitions = optNumPartitions.getOrElse(SQLConf.get.numShufflePartitions) require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") val partitioning: Partitioning = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala index 34baf5b90e..05fc1f7958 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala @@ -79,7 +79,15 @@ object BasicStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { override def visitScriptTransform(p: ScriptTransformation): Statistics = default(p) - override def visitUnion(p: Union): Statistics = fallback(p) + override def visitUnion(p: Union): Statistics = { + val stats = p.children.map(_.stats) + val rowCount = if (stats.exists(_.rowCount.isEmpty)) { + None + } else { + Some(stats.map(_.rowCount.get).sum) + } + Statistics(sizeInBytes = stats.map(_.sizeInBytes).sum, rowCount = rowCount) + } override def visitWindow(p: Window): Statistics = fallback(p) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 780d2bad1b..b4f12db439 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -670,9 +670,10 @@ object DateTimeUtils { private val FRIDAY = 1 private val SATURDAY = 2 - /* + /** * Returns day of week from String. Starting from Thursday, marked as 0. * (Because 1970-01-01 is Thursday). + * @throws IllegalArgumentException if the input is not a valid day of week. */ def getDayOfWeekFromString(string: UTF8String): Int = { val dowString = string.toString.toUpperCase(Locale.ROOT) @@ -684,7 +685,8 @@ object DateTimeUtils { case "TH" | "THU" | "THURSDAY" => THURSDAY case "FR" | "FRI" | "FRIDAY" => FRIDAY case "SA" | "SAT" | "SATURDAY" => SATURDAY - case _ => -1 + case _ => + throw new IllegalArgumentException(s"""Illegal input for day of week: $string""") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala index 08c1b27cae..f411382c1a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExceptionPositionSuite.scala @@ -40,6 +40,23 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { verifyViewPosition("ALTER VIEW unknown AS SELECT 1", "unknown") } + test("SPARK-34057: UnresolvedTableOrView should retain sql text position") { + verifyTableOrViewPosition("DROP TABLE unknown", "unknown") + verifyTableOrViewPosition("DESCRIBE TABLE unknown", "unknown") + verifyTableOrPermanentViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS", "unknown") + verifyTableOrViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS FOR COLUMNS col", "unknown") + verifyTableOrViewPosition("ANALYZE TABLE unknown COMPUTE STATISTICS FOR ALL COLUMNS", "unknown") + verifyTableOrPermanentViewPosition("SHOW CREATE TABLE unknown", "unknown") + verifyTableOrViewPosition("REFRESH TABLE unknown", "unknown") + verifyTableOrViewPosition("SHOW COLUMNS FROM unknown", "unknown") + // Special case where namespace is prepended to the table name. + assertAnalysisError( + parsePlan("SHOW COLUMNS FROM unknown IN db"), + Seq(s"Table or view not found: db.unknown; line 1 pos 18")) + verifyTableOrViewPosition("ALTER TABLE unknown RENAME TO t", "unknown") + verifyTableOrViewPosition("ALTER VIEW unknown RENAME TO v", "unknown") + } + private def verifyTablePosition(sql: String, table: String): Unit = { verifyPosition(sql, table, "Table") } @@ -48,6 +65,14 @@ class AnalysisExceptionPositionSuite extends AnalysisTest { verifyPosition(sql, table, "View") } + private def verifyTableOrViewPosition(sql: String, table: String): Unit = { + verifyPosition(sql, table, "Table or view") + } + + private def verifyTableOrPermanentViewPosition(sql: String, table: String): Unit = { + verifyPosition(sql, table, "Table or permanent view") + } + private def verifyPosition(sql: String, table: String, msgPrefix: String): Unit = { val expectedPos = sql.indexOf(table) assert(expectedPos != -1) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index fa779477cc..4867dafa3d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -685,7 +685,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { isMapGroupsWithState = false, null, streamRelation).groupBy("*")(count("*")), OutputMode.Append()) - Seq(Inner, LeftOuter, RightOuter).map { joinType => + Seq(Inner, LeftOuter, RightOuter).foreach { joinType => assertFailOnGlobalWatermarkLimit( s"stream-stream $joinType after FlatMapGroupsWithState in Append mode", streamRelation.join( @@ -718,7 +718,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { Deduplicate(Seq(attribute), streamRelation).groupBy("a")(count("*")), OutputMode.Append()) - Seq(Inner, LeftOuter, RightOuter).map { joinType => + Seq(Inner, LeftOuter, RightOuter).foreach { joinType => assertPassOnGlobalWatermarkLimit( s"$joinType join after deduplicate in Append mode", streamRelation.join(Deduplicate(Seq(attribute), streamRelation), joinType = joinType, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 79770505ec..1af8fe8828 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -640,13 +640,33 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { testNextDay("2015-07-23", "Fri", "2015-07-24") testNextDay("2015-07-23", "fr", "2015-07-24") - checkEvaluation(NextDay(Literal(Date.valueOf("2015-07-23")), Literal("xx")), null) - checkEvaluation(NextDay(Literal.create(null, DateType), Literal("xx")), null) - checkEvaluation( - NextDay(Literal(Date.valueOf("2015-07-23")), Literal.create(null, StringType)), null) - // Test escaping of dayOfWeek - GenerateUnsafeProjection.generate( - NextDay(Literal(Date.valueOf("2015-07-23")), Literal("\"quote")) :: Nil) + Seq(true, false).foreach { ansiEnabled => + withSQLConf(SQLConf.ANSI_ENABLED.key -> ansiEnabled.toString) { + var expr: Expression = NextDay(Literal(Date.valueOf("2015-07-23")), Literal("xx")) + if (ansiEnabled) { + val errMsg = "Illegal input for day of week: xx" + checkExceptionInExpression[Exception](expr, errMsg) + } else { + checkEvaluation(expr, null) + } + + expr = NextDay(Literal.create(null, DateType), Literal("xx")) + checkEvaluation(expr, null) + + expr = NextDay(Literal(Date.valueOf("2015-07-23")), Literal.create(null, StringType)) + checkEvaluation(expr, null) + + // Test escaping of dayOfWeek + expr = NextDay(Literal(Date.valueOf("2015-07-23")), Literal("\"quote")) + GenerateUnsafeProjection.generate(expr :: Nil) + if (ansiEnabled) { + val errMsg = """Illegal input for day of week: "quote""" + checkExceptionInExpression[Exception](expr, errMsg) + } else { + checkEvaluation(expr, null) + } + } + } } private def testTruncDate(input: Date, fmt: String, expected: Date): Unit = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala index 1812dce0da..c06c92f9c1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala @@ -164,4 +164,72 @@ class LikeSimplificationSuite extends PlanTest { .analyze comparePlans(optimized5, correctAnswer5) } + + test("simplify LikeAll") { + val originalQuery = + testRelation + .where(('a likeAll( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((StartsWith('a, "abc") && EndsWith('a, "xyz")) && + (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && + Contains('a, "mn")) && ('a === "")) && ('a === "abc")) && + ('a likeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify NotLikeAll") { + val originalQuery = + testRelation + .where(('a notLikeAll( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((Not(StartsWith('a, "abc")) && Not(EndsWith('a, "xyz"))) && + Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) && + Not(Contains('a, "mn"))) && Not('a === "")) && Not('a === "abc")) && + ('a notLikeAll("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify LikeAny") { + val originalQuery = + testRelation + .where(('a likeAny( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((StartsWith('a, "abc") || EndsWith('a, "xyz")) || + (Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || + Contains('a, "mn")) || ('a === "")) || ('a === "abc")) || + ('a likeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify NotLikeAny") { + val originalQuery = + testRelation + .where(('a notLikeAny( + "abc%", "abc\\%", "%xyz", "abc\\%def", "abc%def", "%mn%", "%mn\\%", "", "abc"))) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .where((((((Not(StartsWith('a, "abc")) || Not(EndsWith('a, "xyz"))) || + Not(Length('a) >= 6 && (StartsWith('a, "abc") && EndsWith('a, "def")))) || + Not(Contains('a, "mn"))) || Not('a === "")) || Not('a === "abc")) || + ('a notLikeAny("abc\\%", "abc\\%def", "%mn\\%"))) + .analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala index 1f3c24bdbb..04ebb4e63c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SimplifyConditionalsInPredicateSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{And, CaseWhen, Expression, If, IsNotNull, Literal, Or} +import org.apache.spark.sql.catalyst.expressions.{And, CaseWhen, Expression, If, IsNotNull, Literal, Or, Rand} import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, LocalRelation, LogicalPlan, UpdateTable} @@ -158,6 +158,15 @@ class SimplifyConditionalsInPredicateSuite extends PlanTest { testProjection(originalCond, expectedExpr = originalCond) } + test("CASE WHEN non-deterministic-cond THEN false END") { + val originalCond = + CaseWhen(Seq((UnresolvedAttribute("i") > Rand(0), FalseLiteral))) + val expectedCond = And(UnresolvedAttribute("i") > Rand(0), FalseLiteral) + // nondeterministic expressions are only allowed in Project, Filter, Aggregate or Window, + testFilter(originalCond, expectedCond = FalseLiteral) + testProjection(originalCond, expectedExpr = originalCond) + } + test("CASE WHEN cond THEN true ELSE elseVal END => OR(cond, elseVal)") { val originalCond = CaseWhen( Seq((UnresolvedAttribute("i") > Literal(10), TrueLiteral)), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala index b84207397e..2e1cf4a137 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/JoinReorderSuite.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.optimizer._ +import org.apache.spark.sql.catalyst.optimizer.JoinReorderDP.JoinPlan import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -363,4 +364,18 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB assertEqualJoinPlans(Optimize, originalPlan3, bestPlan3) } + + test("SPARK-33935: betterThan should be consistent") { + val plan1 = JoinPlan(null, null, null, Cost(300, 80)) + val plan2 = JoinPlan(null, null, null, Cost(500, 30)) + + // cost1 = 300*0.7 + 80*0.3 = 234 + // cost2 = 500*0.7 + 30*0.3 = 359 + + assert(!plan1.betterThan(plan1, conf)) + assert(!plan2.betterThan(plan2, conf)) + + assert(plan1.betterThan(plan2, conf)) + assert(!plan2.betterThan(plan1, conf)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala index 703be48c6a..a42914765d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/joinReorder/StarJoinCostBasedReorderSuite.scala @@ -294,12 +294,12 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - f1.join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) - .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) - .join(t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))), Inner, - Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .join(f1 + .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) + .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk")))) .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualJoinPlans(Optimize, query, expected) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index 9ec22a982a..b0640ecc4e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -1089,13 +1089,13 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ALTER TABLE a.b.c RENAME TO x.y.z"), RenameTable( - UnresolvedTableOrView(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO"), + UnresolvedTableOrView(Seq("a", "b", "c"), "ALTER TABLE ... RENAME TO", true), Seq("x", "y", "z"), isView = false)) comparePlans( parsePlan("ALTER VIEW a.b.c RENAME TO x.y.z"), RenameTable( - UnresolvedTableOrView(Seq("a", "b", "c"), "ALTER VIEW ... RENAME TO"), + UnresolvedTableOrView(Seq("a", "b", "c"), "ALTER VIEW ... RENAME TO", true), Seq("x", "y", "z"), isView = true)) } @@ -1103,38 +1103,38 @@ class DDLParserSuite extends AnalysisTest { test("describe table column") { comparePlans(parsePlan("DESCRIBE t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("col")), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `abc.xyz`"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("abc.xyz")), isExtended = false)) comparePlans(parsePlan("DESCRIBE t abc.xyz"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("abc", "xyz")), isExtended = false)) comparePlans(parsePlan("DESCRIBE t `a.b`.`x.y`"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("a.b", "x.y")), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("col")), isExtended = false)) comparePlans(parsePlan("DESCRIBE TABLE EXTENDED t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("col")), isExtended = true)) comparePlans(parsePlan("DESCRIBE TABLE FORMATTED t col"), DescribeColumn( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), UnresolvedAttribute(Seq("col")), isExtended = true)) @@ -1156,16 +1156,16 @@ class DDLParserSuite extends AnalysisTest { test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { comparePlans(parsePlan("describe t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = false)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = false)) comparePlans(parsePlan("describe table extended t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = true)) comparePlans(parsePlan("describe table formatted t"), DescribeRelation( - UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE"), Map.empty, isExtended = true)) + UnresolvedTableOrView(Seq("t"), "DESCRIBE TABLE", true), Map.empty, isExtended = true)) } test("insert table: basic append") { @@ -1807,40 +1807,6 @@ class DDLParserSuite extends AnalysisTest { UnresolvedNamespace(Seq("a", "b", "c")), "/home/user/db")) } - test("show databases: basic") { - comparePlans( - parsePlan("SHOW DATABASES"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) - comparePlans( - parsePlan("SHOW DATABASES LIKE 'defau*'"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), Some("defau*"))) - } - - test("show databases: FROM/IN operator is not allowed") { - def verify(sql: String): Unit = { - val exc = intercept[ParseException] { parsePlan(sql) } - assert(exc.getMessage.contains("FROM/IN operator is not allowed in SHOW DATABASES")) - } - - verify("SHOW DATABASES FROM testcat.ns1.ns2") - verify("SHOW DATABASES IN testcat.ns1.ns2") - } - - test("show namespaces") { - comparePlans( - parsePlan("SHOW NAMESPACES"), - ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) - comparePlans( - parsePlan("SHOW NAMESPACES FROM testcat.ns1.ns2"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) - comparePlans( - parsePlan("SHOW NAMESPACES IN testcat.ns1.ns2"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) - comparePlans( - parsePlan("SHOW NAMESPACES IN testcat.ns1 LIKE '*pattern*'"), - ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1")), Some("*pattern*"))) - } - test("analyze table statistics") { comparePlans(parsePlan("analyze table a.b.c compute statistics"), AnalyzeTable( @@ -1909,7 +1875,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans( parsePlan("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR COLUMNS key, value"), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ...", true), Option(Seq("key", "value")), allColumns = false)) @@ -1921,7 +1887,7 @@ class DDLParserSuite extends AnalysisTest { |COMPUTE STATISTICS FOR COLUMNS key, value """.stripMargin), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ..."), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR COLUMNS ...", true), Option(Seq("key", "value")), allColumns = false)) @@ -1933,7 +1899,7 @@ class DDLParserSuite extends AnalysisTest { |COMPUTE STATISTICS FOR ALL COLUMNS """.stripMargin), AnalyzeColumn( - UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR ALL COLUMNS"), + UnresolvedTableOrView(Seq("a", "b", "c"), "ANALYZE TABLE ... FOR ALL COLUMNS", true), None, allColumns = true)) @@ -2045,7 +2011,7 @@ class DDLParserSuite extends AnalysisTest { test("REFRESH TABLE") { comparePlans( parsePlan("REFRESH TABLE a.b.c"), - RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c"), "REFRESH TABLE"))) + RefreshTable(UnresolvedTableOrView(Seq("a", "b", "c"), "REFRESH TABLE", true))) } test("show columns") { @@ -2055,15 +2021,15 @@ class DDLParserSuite extends AnalysisTest { val sql4 = "SHOW COLUMNS FROM db1.t1 IN db1" val parsed1 = parsePlan(sql1) - val expected1 = ShowColumns(UnresolvedTableOrView(Seq("t1"), "SHOW COLUMNS"), None) + val expected1 = ShowColumns(UnresolvedTableOrView(Seq("t1"), "SHOW COLUMNS", true), None) val parsed2 = parsePlan(sql2) - val expected2 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), None) + val expected2 = ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS", true), None) val parsed3 = parsePlan(sql3) val expected3 = - ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), Some(Seq("db1"))) + ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS", true), Some(Seq("db1"))) val parsed4 = parsePlan(sql4) val expected4 = - ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS"), Some(Seq("db1"))) + ShowColumns(UnresolvedTableOrView(Seq("db1", "t1"), "SHOW COLUMNS", true), Some(Seq("db1"))) comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) @@ -2304,12 +2270,14 @@ class DDLParserSuite extends AnalysisTest { test("SHOW TBLPROPERTIES table") { comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c"), - ShowTableProperties(UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), None)) + ShowTableProperties( + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), + None)) comparePlans( parsePlan("SHOW TBLPROPERTIES a.b.c('propKey1')"), ShowTableProperties( - UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES"), Some("propKey1"))) + UnresolvedTableOrView(Seq("a", "b", "c"), "SHOW TBLPROPERTIES", true), Some("propKey1"))) } test("DESCRIBE FUNCTION") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala index a4d1b5d5e6..5e7adaa716 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala @@ -94,11 +94,11 @@ class ParserUtilsSuite extends SparkFunSuite { assert(unescapeSQLString(""""\256"""") == "256") // String including a '\u0000' style literal characters (\u732B is a cat in Kanji). - assert(unescapeSQLString("\"How cute \u732B are\"") == "How cute \u732B are") + assert(unescapeSQLString("\"How cute \\u732B are\"") == "How cute \u732B are") // String including a surrogate pair character // (\uD867\uDE3D is Okhotsk atka mackerel in Kanji). - assert(unescapeSQLString("\"\uD867\uDE3D is a fish\"") == "\uD867\uDE3D is a fish") + assert(unescapeSQLString("\"\\uD867\\uDE3D is a fish\"") == "\uD867\uDE3D is a fish") // scalastyle:on nonascii } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 91f8fc406a..1d780142ae 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -141,6 +141,17 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { expectedStatsCboOff = Statistics(sizeInBytes = 120)) } + test("SPARK-34031: Union operator missing rowCount when enable CBO") { + val union = Union(plan :: plan :: plan :: Nil) + val childrenSize = union.children.size + val sizeInBytes = plan.size.get * childrenSize + val rowCount = Some(plan.rowCount * childrenSize) + checkStats( + union, + expectedStatsCboOn = Statistics(sizeInBytes = sizeInBytes, rowCount = rowCount), + expectedStatsCboOff = Statistics(sizeInBytes = sizeInBytes)) + } + /** Check estimated stats when cbo is turned on/off. */ private def checkStats( plan: LogicalPlan, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 3d841f3237..b9b55da5a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -675,4 +675,11 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { assert(toDate("tomorrow CET ", zoneId).get === today + 1) } } + + test("parsing day of week") { + assert(getDayOfWeekFromString(UTF8String.fromString("THU")) == 0) + assert(getDayOfWeekFromString(UTF8String.fromString("MONDAY")) == 4) + intercept[IllegalArgumentException](getDayOfWeekFromString(UTF8String.fromString("xx"))) + intercept[IllegalArgumentException](getDayOfWeekFromString(UTF8String.fromString("\"quote"))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala index a1253dfe67..27561857c1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/InMemoryTable.scala @@ -274,11 +274,13 @@ class InMemoryTable( this } - override def buildForBatch(): BatchWrite = writer + override def build(): Write = new Write { + override def toBatch: BatchWrite = writer - override def buildForStreaming(): StreamingWrite = streamingWriter match { - case exc: StreamingNotSupportedOperation => exc.throwsException() - case s => s + override def toStreaming: StreamingWrite = streamingWriter match { + case exc: StreamingNotSupportedOperation => exc.throwsException() + case s => s + } } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 7a9f61a2cc..42ceebccdd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -165,7 +165,8 @@ public UnsafeKVExternalSorter( (int) (long) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, numElementsForSpillThreshold, - inMemSorter); + inMemSorter, + map.getTotalMemoryConsumption()); // reset the map, so we can re-use it to insert new records. the inMemSorter will not used // anymore, so the underline array could be used by map again. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala index b66f94ae11..669b90f4d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/BaseScriptTransformationExec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import java.io._ +import java.io.{BufferedReader, File, InputStream, InputStreamReader, OutputStream} import java.nio.charset.StandardCharsets import java.util.concurrent.TimeUnit diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index b3671945e5..0c6f22dea7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -221,7 +221,7 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper { // Remove the cache entry before creating a new ones. cachedData = cachedData.filterNot(cd => needToRecache.exists(_ eq cd)) } - needToRecache.map { cd => + needToRecache.foreach { cd => cd.cachedRepresentation.cacheBuilder.clearCache() val sessionWithConfigsOff = SparkSession.getOrCloneSessionWithConfigsOff( spark, forceDisableConfigs) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 5e3a67927e..4545d730a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -485,6 +485,7 @@ case class AlterTableAddPartitionCommand( catalog.createPartitions(table.identifier, batch, ignoreIfExists = ifNotExists) } + sparkSession.catalog.refreshTable(table.identifier.quotedString) if (table.stats.nonEmpty) { if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { val addedSize = CommandUtils.calculateMultipleLocationSizes(sparkSession, table.identifier, @@ -536,6 +537,7 @@ case class AlterTableRenamePartitionCommand( catalog.renamePartitions( tableName, Seq(normalizedOldPartition), Seq(normalizedNewPartition)) + sparkSession.catalog.refreshTable(table.identifier.quotedString) Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index b9866e415c..4fd6684b3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -406,7 +406,8 @@ object PreprocessTableInsertion extends Rule[LogicalPlan] { catalogTable.get.tracksPartitionsInCatalog if (partitionsTrackedByCatalog && normalizedPartSpec.nonEmpty) { // empty partition column value - if (normalizedPartSpec.filter(_._2.isDefined).exists(_._2.get.isEmpty)) { + if (normalizedPartSpec.map(_._2) + .filter(_.isDefined).map(_.get).exists(v => v != null && v.isEmpty)) { val spec = normalizedPartSpec.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]") throw new AnalysisException( s"Partition spec is invalid. The spec ($spec) contains an empty partition column value") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala index 38b83e3ad7..0632bd7510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableRenamePartitionExec.scala @@ -28,12 +28,15 @@ import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement case class AlterTableRenamePartitionExec( table: SupportsPartitionManagement, from: ResolvedPartitionSpec, - to: ResolvedPartitionSpec) extends V2CommandExec { + to: ResolvedPartitionSpec, + refreshCache: () => Unit) extends V2CommandExec { override def output: Seq[Attribute] = Seq.empty override protected def run(): Seq[InternalRow] = { - table.renamePartition(from.ident, to.ident) + if (table.renamePartition(from.ident, to.ident)) { + refreshCache() + } Seq.empty } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 1537ebf8f3..028a2fc690 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression, Na import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.toPrettySQL -import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalog, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, TableCapability, TableCatalog, TableChange} +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, StagingTableCatalog, SupportsNamespaces, SupportsPartitionManagement, SupportsWrite, Table, TableCapability, TableCatalog, TableChange} import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.connector.write.V1Write import org.apache.spark.sql.errors.QueryCompilationErrors @@ -81,6 +81,11 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat } } + private def invalidateCache(catalog: TableCatalog, table: Table, ident: Identifier): Unit = { + val v2Relation = DataSourceV2Relation.create(table, Some(catalog), Some(ident)) + session.sharedState.cacheManager.uncacheQuery(session, v2Relation, cascade = true) + } + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(project, filters, relation @ DataSourceV2ScanRelation(_, V1ScanWrapper(scan, translated, pushed), output)) => @@ -164,10 +169,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat catalog match { case staging: StagingTableCatalog => AtomicReplaceTableExec( - staging, ident, schema, parts, propsWithOwner, orCreate = orCreate) :: Nil + staging, ident, schema, parts, propsWithOwner, orCreate = orCreate, + invalidateCache) :: Nil case _ => ReplaceTableExec( - catalog, ident, schema, parts, propsWithOwner, orCreate = orCreate) :: Nil + catalog, ident, schema, parts, propsWithOwner, orCreate = orCreate, + invalidateCache) :: Nil } case ReplaceTableAsSelect(catalog, ident, parts, query, props, options, orCreate) => @@ -176,7 +183,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat catalog match { case staging: StagingTableCatalog => AtomicReplaceTableAsSelectExec( - session, staging, ident, parts, @@ -184,10 +190,10 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat planLater(query), propsWithOwner, writeOptions, - orCreate = orCreate) :: Nil + orCreate = orCreate, + invalidateCache) :: Nil case _ => ReplaceTableAsSelectExec( - session, catalog, ident, parts, @@ -195,7 +201,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat planLater(query), propsWithOwner, writeOptions, - orCreate = orCreate) :: Nil + orCreate = orCreate, + invalidateCache) :: Nil } case AppendData(r @ DataSourceV2Relation(v1: SupportsWrite, _, _, _, _), query, writeOptions, @@ -368,11 +375,12 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat invalidateCache(r, recacheTable = true)) :: Nil case AlterTableRenamePartition( - ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => + r @ ResolvedTable(_, _, table: SupportsPartitionManagement, _), from, to) => AlterTableRenamePartitionExec( table, Seq(from).asResolvedPartitionSpecs.head, - Seq(to).asResolvedPartitionSpecs.head) :: Nil + Seq(to).asResolvedPartitionSpecs.head, + invalidateCache(r, recacheTable = true)) :: Nil case AlterTableRecoverPartitions(_: ResolvedTable) => throw new AnalysisException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala index c7ce69f744..ab8c5617aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeColumnExec.scala @@ -20,17 +20,12 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.catalyst.expressions.Attribute case class DescribeColumnExec( override val output: Seq[Attribute], column: Attribute, isExtended: Boolean) extends V2CommandExec { - private val toRow = { - RowEncoder(StructType.fromAttributes(output)).resolveAndBind().createSerializer() - } override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() @@ -49,8 +44,4 @@ case class DescribeColumnExec( rows.toSeq } - - private def toCatalystRow(strs: String*): InternalRow = { - toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala index e273abf90e..2da96b769a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala @@ -21,10 +21,8 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} -import org.apache.spark.sql.types.StructType /** * Physical plan node for describing a namespace. @@ -34,10 +32,6 @@ case class DescribeNamespaceExec( catalog: SupportsNamespaces, namespace: Seq[String], isExtended: Boolean) extends V2CommandExec { - private val toRow = { - RowEncoder(StructType.fromAttributes(output)).resolveAndBind().createSerializer() - } - override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() val ns = namespace.toArray @@ -57,8 +51,4 @@ case class DescribeNamespaceExec( } rows.toSeq } - - private def toCatalystRow(strs: String*): InternalRow = { - toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala index 0ca442baee..769d76a9b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeTableExec.scala @@ -21,20 +21,13 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsMetadataColumns, Table} -import org.apache.spark.sql.types.StructType case class DescribeTableExec( output: Seq[Attribute], table: Table, isExtended: Boolean) extends V2CommandExec { - - private val toRow = { - RowEncoder(StructType.fromAttributes(output)).resolveAndBind().createSerializer() - } - override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() addSchema(rows) @@ -99,8 +92,4 @@ case class DescribeTableExec( } private def emptyRow(): InternalRow = toCatalystRow("", "", "") - - private def toCatalystRow(strs: String*): InternalRow = { - toRow(new GenericRowWithSchema(strs.toArray, schema)).copy() - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala index 1f3bcf2e3f..10c09f4be7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceTableExec.scala @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, TableCatalog} +import org.apache.spark.sql.connector.catalog.{Identifier, StagedTable, StagingTableCatalog, Table, TableCatalog} import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -33,10 +33,13 @@ case class ReplaceTableExec( tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], - orCreate: Boolean) extends V2CommandExec { + orCreate: Boolean, + invalidateCache: (TableCatalog, Table, Identifier) => Unit) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { if (catalog.tableExists(ident)) { + val table = catalog.loadTable(ident) + invalidateCache(catalog, table, ident) catalog.dropTable(ident) } else if (!orCreate) { throw new CannotReplaceMissingTableException(ident) @@ -54,9 +57,14 @@ case class AtomicReplaceTableExec( tableSchema: StructType, partitioning: Seq[Transform], tableProperties: Map[String, String], - orCreate: Boolean) extends V2CommandExec { + orCreate: Boolean, + invalidateCache: (TableCatalog, Table, Identifier) => Unit) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { + if (catalog.tableExists(identifier)) { + val table = catalog.loadTable(identifier) + invalidateCache(catalog, table, identifier) + } val staged = if (orCreate) { catalog.stageCreateOrReplace( identifier, tableSchema, partitioning.toArray, tableProperties.asJava) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala index 9e6f00e092..b13cea2667 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetCatalogAndNamespaceExec.scala @@ -32,8 +32,8 @@ case class SetCatalogAndNamespaceExec( override protected def run(): Seq[InternalRow] = { // The catalog is updated first because CatalogManager resets the current namespace // when the current catalog is set. - catalogName.map(catalogManager.setCurrentCatalog) - namespace.map(ns => catalogManager.setCurrentNamespace(ns.toArray)) + catalogName.foreach(catalogManager.setCurrentCatalog) + namespace.foreach(ns => catalogManager.setCurrentNamespace(ns.toArray)) Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCurrentNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCurrentNamespaceExec.scala index 5f7b6f4061..121ae1c5b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCurrentNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCurrentNamespaceExec.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper @@ -31,11 +30,6 @@ case class ShowCurrentNamespaceExec( catalogManager: CatalogManager) extends V2CommandExec { override protected def run(): Seq[InternalRow] = { - val toRow = RowEncoder(schema).resolveAndBind().createSerializer() - val result = new GenericRowWithSchema(Array[Any]( - catalogManager.currentCatalog.name, - catalogManager.currentNamespace.quoted), - schema) - Seq(toRow(result).copy()) + Seq(toCatalystRow(catalogManager.currentCatalog.name, catalogManager.currentNamespace.quoted)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala index ceeed0f840..9dafbd79a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowNamespacesExec.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.SupportsNamespaces @@ -44,11 +43,9 @@ case class ShowNamespacesExec( } val rows = new ArrayBuffer[InternalRow]() - val toRow = RowEncoder(schema).resolveAndBind().createSerializer() - namespaces.map(_.quoted).map { ns => if (pattern.map(StringUtils.filterPattern(Seq(ns), _).nonEmpty).getOrElse(true)) { - rows += toRow(new GenericRowWithSchema(Array(ns), schema)).copy() + rows += toCatalystRow(ns) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala index 6d3a94ef15..4e1633e146 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablePropertiesExec.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Table} /** @@ -32,19 +31,18 @@ case class ShowTablePropertiesExec( override protected def run(): Seq[InternalRow] = { import scala.collection.JavaConverters._ - val toRow = RowEncoder(schema).resolveAndBind().createSerializer() // The reserved properties are accessible through DESCRIBE val properties = catalogTable.properties.asScala - .filter { case (k, v) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) } + .filter { case (k, _) => !CatalogV2Util.TABLE_RESERVED_PROPERTIES.contains(k) } propertyKey match { case Some(p) => val propValue = properties .getOrElse(p, s"Table ${catalogTable.name} does not have property: $p") - Seq(toRow(new GenericRowWithSchema(Array(p, propValue), schema)).copy()) + Seq(toCatalystRow(p, propValue)) case None => properties.keys.map(k => - toRow(new GenericRowWithSchema(Array(k, properties(k)), schema)).copy()).toSeq + toCatalystRow(k, properties(k))).toSeq } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala index 5ba01deae9..7ada8d2e5c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowTablesExec.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRowWithSchema} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.NamespaceHelper import org.apache.spark.sql.connector.catalog.TableCatalog @@ -37,15 +36,11 @@ case class ShowTablesExec( pattern: Option[String]) extends V2CommandExec with LeafExecNode { override protected def run(): Seq[InternalRow] = { val rows = new ArrayBuffer[InternalRow]() - val toRow = RowEncoder(schema).resolveAndBind().createSerializer() val tables = catalog.listTables(namespace.toArray) tables.map { table => if (pattern.map(StringUtils.filterPattern(Seq(table.name()), _).nonEmpty).getOrElse(true)) { - val result = new GenericRowWithSchema( - Array(table.namespace().quoted, table.name()), - schema) - rows += toRow(result).copy() + rows += toCatalystRow(table.namespace().quoted, table.name()) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala index 6b193674cc..b54c46fc15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2CommandExec.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.AttributeSet +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, GenericRowWithSchema} import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.StructType /** * A physical operator that executes run() and saves the result to prevent multiple executions. @@ -59,4 +61,11 @@ abstract class V2CommandExec extends SparkPlan { override def producedAttributes: AttributeSet = outputSet + protected def toCatalystRow(strs: String*): InternalRow = { + rowSerializer(new GenericRowWithSchema(strs.toArray, schema)).copy() + } + + private lazy val rowSerializer = { + RowEncoder(StructType.fromAttributes(output)).resolveAndBind().createSerializer() + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index fea8bd25f5..5fa091ea4e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -26,7 +26,6 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.expressions.Attribute @@ -130,7 +129,6 @@ case class AtomicCreateTableAsSelectExec( * ReplaceTableAsSelectStagingExec. */ case class ReplaceTableAsSelectExec( - session: SparkSession, catalog: TableCatalog, ident: Identifier, partitioning: Seq[Transform], @@ -138,7 +136,8 @@ case class ReplaceTableAsSelectExec( query: SparkPlan, properties: Map[String, String], writeOptions: CaseInsensitiveStringMap, - orCreate: Boolean) extends TableWriteExecHelper { + orCreate: Boolean, + invalidateCache: (TableCatalog, Table, Identifier) => Unit) extends TableWriteExecHelper { override protected def run(): Seq[InternalRow] = { // Note that this operation is potentially unsafe, but these are the strict semantics of @@ -151,7 +150,7 @@ case class ReplaceTableAsSelectExec( // 3. The table returned by catalog.createTable doesn't support writing. if (catalog.tableExists(ident)) { val table = catalog.loadTable(ident) - uncacheTable(session, catalog, table, ident) + invalidateCache(catalog, table, ident) catalog.dropTable(ident) } else if (!orCreate) { throw new CannotReplaceMissingTableException(ident) @@ -176,7 +175,6 @@ case class ReplaceTableAsSelectExec( * is left untouched. */ case class AtomicReplaceTableAsSelectExec( - session: SparkSession, catalog: StagingTableCatalog, ident: Identifier, partitioning: Seq[Transform], @@ -184,13 +182,14 @@ case class AtomicReplaceTableAsSelectExec( query: SparkPlan, properties: Map[String, String], writeOptions: CaseInsensitiveStringMap, - orCreate: Boolean) extends TableWriteExecHelper { + orCreate: Boolean, + invalidateCache: (TableCatalog, Table, Identifier) => Unit) extends TableWriteExecHelper { override protected def run(): Seq[InternalRow] = { val schema = CharVarcharUtils.getRawSchema(query.schema).asNullable if (catalog.tableExists(ident)) { val table = catalog.loadTable(ident) - uncacheTable(session, catalog, table, ident) + invalidateCache(catalog, table, ident) } val staged = if (orCreate) { catalog.stageCreateOrReplace( @@ -364,15 +363,6 @@ trait V2TableWriteExec extends V2CommandExec with UnaryExecNode { Nil } - - protected def uncacheTable( - session: SparkSession, - catalog: TableCatalog, - table: Table, - ident: Identifier): Unit = { - val plan = DataSourceV2Relation.create(table, Some(catalog), Some(ident)) - session.sharedState.cacheManager.uncacheQuery(session, plan, cascade = true) - } } object DataWritingSparkTask extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala index 27558e5b0d..c4258fa264 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalog.scala @@ -127,7 +127,7 @@ class JDBCTableCatalog extends TableCatalog with SupportsNamespaces with Logging var tableComment: String = "" var tableProperties: String = "" if (!properties.isEmpty) { - properties.asScala.map { + properties.asScala.foreach { case (k, v) => k match { case TableCatalog.PROP_COMMENT => tableComment = v case TableCatalog.PROP_PROVIDER => @@ -226,7 +226,7 @@ class JDBCTableCatalog extends TableCatalog with SupportsNamespaces with Logging case Array(db) if !namespaceExists(namespace) => var comment = "" if (!metadata.isEmpty) { - metadata.asScala.map { + metadata.asScala.foreach { case (k, v) => k match { case SupportsNamespaces.PROP_COMMENT => comment = v case SupportsNamespaces.PROP_OWNER => // ignore diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index c9f40fa22b..67803ad76d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -627,21 +627,22 @@ abstract class StreamExecution( inputPlan.schema, new CaseInsensitiveStringMap(options.asJava)) val writeBuilder = table.newWriteBuilder(info) - outputMode match { + val write = outputMode match { case Append => - writeBuilder.buildForStreaming() + writeBuilder.build() case Complete => // TODO: we should do this check earlier when we have capability API. require(writeBuilder.isInstanceOf[SupportsTruncate], table.name + " does not support Complete mode.") - writeBuilder.asInstanceOf[SupportsTruncate].truncate().buildForStreaming() + writeBuilder.asInstanceOf[SupportsTruncate].truncate().build() case Update => require(writeBuilder.isInstanceOf[SupportsStreamingUpdateAsAppend], table.name + " does not support Update mode.") - writeBuilder.asInstanceOf[SupportsStreamingUpdateAsAppend].buildForStreaming() + writeBuilder.asInstanceOf[SupportsStreamingUpdateAsAppend].build() } + write.toStreaming } protected def purge(threshold: Long): Unit = { diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index acfd1f50e1..0493d8653c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -172,3 +172,10 @@ select to_unix_timestamp("2020-01-27T20:06:11.847", "yyyy-MM-dd HH:mm:ss.SSS"); select to_unix_timestamp("Unparseable", "yyyy-MM-dd HH:mm:ss.SSS"); select cast("Unparseable" as timestamp); select cast("Unparseable" as date); + +-- next_day +select next_day("2015-07-23", "Mon"); +select next_day("2015-07-23", "xx"); +select next_day("xx", "Mon"); +select next_day(null, "Mon"); +select next_day(null, "xx"); diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 81e2204358..6ee1014739 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -86,6 +86,16 @@ SELECT 1 FROM range(10) HAVING MAX(id) > 0; SELECT id FROM range(10) HAVING id > 0; +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true; + +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; + +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false; + -- Test data CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out index 3e307a92c1..9a0c8ff02c 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 123 +-- Number of queries: 128 -- !query @@ -1069,3 +1069,45 @@ struct<> -- !query output java.time.DateTimeException Cannot cast Unparseable to DateType. + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct<> +-- !query output +java.lang.IllegalArgumentException +Illegal input for day of week: xx + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct<> +-- !query output +java.time.DateTimeException +Cannot cast xx to DateType. + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out index ed54b72111..d93843b231 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime-legacy.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 123 +-- Number of queries: 128 -- !query @@ -1021,3 +1021,43 @@ select cast("Unparseable" as date) struct -- !query output NULL + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 213895dcb4..b07b68ce26 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 123 +-- Number of queries: 128 -- !query @@ -1029,3 +1029,43 @@ select cast("Unparseable" as date) struct -- !query output NULL + + +-- !query +select next_day("2015-07-23", "Mon") +-- !query schema +struct +-- !query output +2015-07-27 + + +-- !query +select next_day("2015-07-23", "xx") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day("xx", "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "Mon") +-- !query schema +struct +-- !query output +NULL + + +-- !query +select next_day(null, "xx") +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 75bda87b37..cc07cd64f3 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 57 +-- Number of queries: 62 -- !query @@ -277,6 +277,67 @@ org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get. +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=true +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere true + + +-- !query +SELECT 1 FROM range(10) HAVING true +-- !query schema +struct<1:int> +-- !query output +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 + + +-- !query +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(max(`id`) > CAST(0 AS BIGINT))] +Invalid expressions: [max(`id`)] + + +-- !query +SELECT id FROM range(10) HAVING id > 0 +-- !query schema +struct +-- !query output +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query +SET spark.sql.legacy.parser.havingWithoutGroupByAsWhere=false +-- !query schema +struct +-- !query output +spark.sql.legacy.parser.havingWithoutGroupByAsWhere false + + -- !query CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES (1, true), (1, false), diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out index 3535b30d29..d8d0926d24 100644 --- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -94,7 +94,7 @@ SHOW COLUMNS IN badtable FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.badtable; line 1 pos 0 +Table or view not found: showdb.badtable; line 1 pos 16 -- !query @@ -130,7 +130,7 @@ SHOW COLUMNS IN showdb.showcolumn3 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 0 +Table or view not found: showdb.showcolumn3; line 1 pos 16 -- !query @@ -139,7 +139,7 @@ SHOW COLUMNS IN showcolumn3 FROM showdb struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showdb.showcolumn3; line 1 pos 0 +Table or view not found: showdb.showcolumn3; line 1 pos 16 -- !query @@ -148,7 +148,7 @@ SHOW COLUMNS IN showcolumn4 struct<> -- !query output org.apache.spark.sql.AnalysisException -Table or view not found: showcolumn4; line 1 pos 0 +Table or view not found: showcolumn4; line 1 pos 16 -- !query diff --git a/sql/core/src/test/resources/test_script.py b/sql/core/src/test/resources/test_script.py index 75b4f106d3..4fcd483f44 100644 --- a/sql/core/src/test/resources/test_script.py +++ b/sql/core/src/test/resources/test_script.py @@ -1,4 +1,4 @@ -#! /usr/bin/python +#!/usr/bin/env python3 # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt index 8ee427262b..327e7db702 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/explain.txt @@ -4,8 +4,8 @@ +- * HashAggregate (36) +- * Project (35) +- * BroadcastHashJoin Inner BuildRight (34) - :- * Project (28) - : +- * BroadcastHashJoin Inner BuildRight (27) + :- * Project (29) + : +- * BroadcastHashJoin Inner BuildRight (28) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.date_dim (16) - : +- BroadcastExchange (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.store (23) + : : +- Scan parquet default.customer_address (16) + : +- BroadcastExchange (27) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.date_dim (23) +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.customer_address (29) + +- * Filter (32) + +- * ColumnarToRow (31) + +- Scan parquet default.store (30) (1) Scan parquet default.store_sales @@ -107,94 +107,94 @@ Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = A Output [7]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] Input [13]: [ss_sold_date_sk#1, ss_hdemo_sk#3, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_sales_price#7, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, cd_marital_status#12, cd_education_status#13, hd_demo_sk#15, hd_dep_count#16] -(16) Scan parquet default.date_dim -Output [2]: [d_date_sk#18, d_year#19] +(16) Scan parquet default.customer_address +Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (18) Filter [codegen id : 3] -Input [2]: [d_date_sk#18, d_year#19] -Condition : ((isnotnull(d_year#19) AND (d_year#19 = 2001)) AND isnotnull(d_date_sk#18)) +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Condition : (((isnotnull(ca_country#20) AND (ca_country#20 = United States)) AND isnotnull(ca_address_sk#18)) AND ((ca_state#19 IN (TX,OH) OR ca_state#19 IN (OR,NM,KY)) OR ca_state#19 IN (VA,TX,MS))) (19) Project [codegen id : 3] -Output [1]: [d_date_sk#18] -Input [2]: [d_date_sk#18, d_year#19] +Output [2]: [ca_address_sk#18, ca_state#19] +Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] (20) BroadcastExchange -Input [1]: [d_date_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +Input [2]: [ca_address_sk#18, ca_state#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] (21) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#18] -Join condition: None +Left keys [1]: [ss_addr_sk#4] +Right keys [1]: [ca_address_sk#18] +Join condition: ((((ca_state#19 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#19 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#19 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) (22) Project [codegen id : 6] -Output [6]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [8]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, d_date_sk#18] +Output [5]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [9]: [ss_sold_date_sk#1, ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#18, ca_state#19] -(23) Scan parquet default.store -Output [1]: [s_store_sk#21] +(23) Scan parquet default.date_dim +Output [2]: [d_date_sk#22, d_year#23] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [1]: [s_store_sk#21] +Input [2]: [d_date_sk#22, d_year#23] (25) Filter [codegen id : 4] -Input [1]: [s_store_sk#21] -Condition : isnotnull(s_store_sk#21) +Input [2]: [d_date_sk#22, d_year#23] +Condition : ((isnotnull(d_year#23) AND (d_year#23 = 2001)) AND isnotnull(d_date_sk#22)) -(26) BroadcastExchange -Input [1]: [s_store_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] +(26) Project [codegen id : 4] +Output [1]: [d_date_sk#22] +Input [2]: [d_date_sk#22, d_year#23] -(27) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#21] +(27) BroadcastExchange +Input [1]: [d_date_sk#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#24] + +(28) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#22] Join condition: None -(28) Project [codegen id : 6] -Output [5]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10] -Input [7]: [ss_addr_sk#4, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, s_store_sk#21] +(29) Project [codegen id : 6] +Output [4]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] +Input [6]: [ss_sold_date_sk#1, ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, d_date_sk#22] -(29) Scan parquet default.customer_address -Output [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(30) Scan parquet default.store +Output [1]: [s_store_sk#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_country), EqualTo(ca_country,United States), IsNotNull(ca_address_sk), Or(Or(In(ca_state, [TX,OH]),In(ca_state, [OR,NM,KY])),In(ca_state, [VA,TX,MS]))] -ReadSchema: struct - -(30) ColumnarToRow [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct -(31) Filter [codegen id : 5] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] -Condition : (((isnotnull(ca_country#25) AND (ca_country#25 = United States)) AND isnotnull(ca_address_sk#23)) AND ((ca_state#24 IN (TX,OH) OR ca_state#24 IN (OR,NM,KY)) OR ca_state#24 IN (VA,TX,MS))) +(31) ColumnarToRow [codegen id : 5] +Input [1]: [s_store_sk#25] -(32) Project [codegen id : 5] -Output [2]: [ca_address_sk#23, ca_state#24] -Input [3]: [ca_address_sk#23, ca_state#24, ca_country#25] +(32) Filter [codegen id : 5] +Input [1]: [s_store_sk#25] +Condition : isnotnull(s_store_sk#25) (33) BroadcastExchange -Input [2]: [ca_address_sk#23, ca_state#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] +Input [1]: [s_store_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_addr_sk#4] -Right keys [1]: [ca_address_sk#23] -Join condition: ((((ca_state#24 IN (TX,OH) AND (ss_net_profit#10 >= 100.00)) AND (ss_net_profit#10 <= 200.00)) OR ((ca_state#24 IN (OR,NM,KY) AND (ss_net_profit#10 >= 150.00)) AND (ss_net_profit#10 <= 300.00))) OR ((ca_state#24 IN (VA,TX,MS) AND (ss_net_profit#10 >= 50.00)) AND (ss_net_profit#10 <= 250.00))) +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#25] +Join condition: None (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [7]: [ss_addr_sk#4, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, ss_net_profit#10, ca_address_sk#23, ca_state#24] +Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] @@ -205,7 +205,7 @@ Results [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] (37) Exchange Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] -Arguments: SinglePartition, true, [id=#41] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#41] (38) HashAggregate [codegen id : 7] Input [7]: [sum#34, count#35, sum#36, count#37, sum#38, count#39, sum#40] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt index b457788dbd..45d6c8f3b0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q13.sf100/simplified.txt @@ -5,11 +5,11 @@ WholeStageCodegen (7) WholeStageCodegen (6) HashAggregate [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] [sum,count,sum,count,sum,count,sum,sum,count,sum,count,sum,count,sum] Project [ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] - BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] - Project [ss_addr_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost] + BroadcastHashJoin [ss_addr_sk,ca_address_sk,ca_state,ss_net_profit] Project [ss_sold_date_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit] BroadcastHashJoin [ss_hdemo_sk,hd_demo_sk,cd_marital_status,cd_education_status,ss_sales_price,hd_dep_count] Project [ss_sold_date_sk,ss_hdemo_sk,ss_addr_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_ext_sales_price,ss_ext_wholesale_cost,ss_net_profit,cd_marital_status,cd_education_status] @@ -35,23 +35,23 @@ WholeStageCodegen (7) InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_year,d_date_sk] + Project [ca_address_sk,ca_state] + Filter [ca_country,ca_address_sk,ca_state] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + Project [d_date_sk] + Filter [d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Project [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt index a17356ae04..a9ab8c3690 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/explain.txt @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (18) Sort [codegen id : 4] Input [5]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#14) (22) Exchange Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(i_item_sk#14, 5), true, [id=#17] +Arguments: hashpartitioning(i_item_sk#14, 5), ENSURE_REQUIREMENTS, [id=#17] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#14, i_item_id#15, i_item_desc#16] @@ -171,7 +171,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s (26) Exchange Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#18] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#18] (27) Sort [codegen id : 8] Input [7]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#19, sr_item_sk#20, sr_customer_sk#21, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), true, [id=#27] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), ENSURE_REQUIREMENTS, [id=#27] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] -Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] -Join condition: None - -(41) Project [codegen id : 12] -Output [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Input [11]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23] - -(42) Exchange -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] -Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, 5), true, [id=#28] - -(43) Sort [codegen id : 13] -Input [7]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23] Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Condition : ((isnotnull(cs_bill_customer_sk#29) AND isnotnull(cs_item_sk#30)) AND isnotnull(cs_sold_date_sk#28)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Condition : ((isnotnull(cs_bill_customer_sk#30) AND isnotnull(cs_item_sk#31)) AND isnotnull(cs_sold_date_sk#29)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#32] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#33] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#28] +Right keys [1]: [d_date_sk#32] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Input [5]: [cs_sold_date_sk#28, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31, d_date_sk#32] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint), 5), ENSURE_REQUIREMENTS, [id=#33] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#29] -Right keys [1]: [d_date_sk#33] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] +Arguments: [cast(cs_bill_customer_sk#29 as bigint) ASC NULLS FIRST, cast(cs_item_sk#30 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] +Right keys [2]: [cast(cs_bill_customer_sk#29 as bigint), cast(cs_item_sk#30 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Input [5]: [cs_sold_date_sk#29, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32, d_date_sk#33] +Output [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Input [7]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_bill_customer_sk#29, cs_item_sk#30, cs_quantity#31] (50) Exchange -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint), 5), true, [id=#34] +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: hashpartitioning(sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22, 5), ENSURE_REQUIREMENTS, [id=#34] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] -Arguments: [cast(cs_bill_customer_sk#30 as bigint) ASC NULLS FIRST, cast(cs_item_sk#31 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] +Arguments: [sr_customer_sk#21 ASC NULLS FIRST, sr_item_sk#20 ASC NULLS FIRST, sr_ticket_number#22 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#21, sr_item_sk#20] -Right keys [2]: [cast(cs_bill_customer_sk#30 as bigint), cast(cs_item_sk#31 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#21, sr_item_sk#20, sr_ticket_number#22] Join condition: None (53) Project [codegen id : 17] -Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] -Input [10]: [ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_return_quantity#23, cs_bill_customer_sk#30, cs_item_sk#31, cs_quantity#32] +Output [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] +Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_quantity#6, s_state#11, i_item_id#15, i_item_desc#16, sr_item_sk#20, sr_customer_sk#21, sr_ticket_number#22, sr_return_quantity#23, cs_quantity#31] (54) HashAggregate [codegen id : 17] -Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#32, s_state#11, i_item_id#15, i_item_desc#16] +Input [6]: [ss_quantity#6, sr_return_quantity#23, cs_quantity#31, s_state#11, i_item_id#15, i_item_desc#16] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#32), partial_avg(cast(cs_quantity#32 as bigint)), partial_stddev_samp(cast(cs_quantity#32 as double))] +Functions [9]: [partial_count(ss_quantity#6), partial_avg(cast(ss_quantity#6 as bigint)), partial_stddev_samp(cast(ss_quantity#6 as double)), partial_count(sr_return_quantity#23), partial_avg(cast(sr_return_quantity#23 as bigint)), partial_stddev_samp(cast(sr_return_quantity#23 as double)), partial_count(cs_quantity#31), partial_avg(cast(cs_quantity#31 as bigint)), partial_stddev_samp(cast(cs_quantity#31 as double))] Aggregate Attributes [18]: [count#35, sum#36, count#37, n#38, avg#39, m2#40, count#41, sum#42, count#43, n#44, avg#45, m2#46, count#47, sum#48, count#49, n#50, avg#51, m2#52] Results [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] (55) Exchange Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] -Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), true, [id=#71] +Arguments: hashpartitioning(i_item_id#15, i_item_desc#16, s_state#11, 5), ENSURE_REQUIREMENTS, [id=#71] (56) HashAggregate [codegen id : 18] Input [21]: [i_item_id#15, i_item_desc#16, s_state#11, count#53, sum#54, count#55, n#56, avg#57, m2#58, count#59, sum#60, count#61, n#62, avg#63, m2#64, count#65, sum#66, count#67, n#68, avg#69, m2#70] Keys [3]: [i_item_id#15, i_item_desc#16, s_state#11] -Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#32), avg(cast(cs_quantity#32 as bigint)), stddev_samp(cast(cs_quantity#32 as double))] -Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#32)#78, avg(cast(cs_quantity#32 as bigint))#79, stddev_samp(cast(cs_quantity#32 as double))#80] -Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#32)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#32 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#32 as double))#80 / avg(cast(cs_quantity#32 as bigint))#79) AS catalog_sales_quantitycov#92] +Functions [9]: [count(ss_quantity#6), avg(cast(ss_quantity#6 as bigint)), stddev_samp(cast(ss_quantity#6 as double)), count(sr_return_quantity#23), avg(cast(sr_return_quantity#23 as bigint)), stddev_samp(cast(sr_return_quantity#23 as double)), count(cs_quantity#31), avg(cast(cs_quantity#31 as bigint)), stddev_samp(cast(cs_quantity#31 as double))] +Aggregate Attributes [9]: [count(ss_quantity#6)#72, avg(cast(ss_quantity#6 as bigint))#73, stddev_samp(cast(ss_quantity#6 as double))#74, count(sr_return_quantity#23)#75, avg(cast(sr_return_quantity#23 as bigint))#76, stddev_samp(cast(sr_return_quantity#23 as double))#77, count(cs_quantity#31)#78, avg(cast(cs_quantity#31 as bigint))#79, stddev_samp(cast(cs_quantity#31 as double))#80] +Results [15]: [i_item_id#15, i_item_desc#16, s_state#11, count(ss_quantity#6)#72 AS store_sales_quantitycount#81, avg(cast(ss_quantity#6 as bigint))#73 AS store_sales_quantityave#82, stddev_samp(cast(ss_quantity#6 as double))#74 AS store_sales_quantitystdev#83, (stddev_samp(cast(ss_quantity#6 as double))#74 / avg(cast(ss_quantity#6 as bigint))#73) AS store_sales_quantitycov#84, count(sr_return_quantity#23)#75 AS as_store_returns_quantitycount#85, avg(cast(sr_return_quantity#23 as bigint))#76 AS as_store_returns_quantityave#86, stddev_samp(cast(sr_return_quantity#23 as double))#77 AS as_store_returns_quantitystdev#87, (stddev_samp(cast(sr_return_quantity#23 as double))#77 / avg(cast(sr_return_quantity#23 as bigint))#76) AS store_returns_quantitycov#88, count(cs_quantity#31)#78 AS catalog_sales_quantitycount#89, avg(cast(cs_quantity#31 as bigint))#79 AS catalog_sales_quantityave#90, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitystdev#91, (stddev_samp(cast(cs_quantity#31 as double))#80 / avg(cast(cs_quantity#31 as bigint))#79) AS catalog_sales_quantitycov#92] (57) TakeOrderedAndProject Input [15]: [i_item_id#15, i_item_desc#16, s_state#11, store_sales_quantitycount#81, store_sales_quantityave#82, store_sales_quantitystdev#83, store_sales_quantitycov#84, as_store_returns_quantitycount#85, as_store_returns_quantityave#86, as_store_returns_quantitystdev#87, store_returns_quantitycov#88, catalog_sales_quantitycount#89, catalog_sales_quantityave#90, catalog_sales_quantitystdev#91, catalog_sales_quantitycov#92] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt index bfb59441f4..79226a34e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q17.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_state,ss_quantity,sr_return_quantity,cs_quantity] [count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2,count,sum,count,n,avg,m2] Project [ss_quantity,sr_return_quantity,cs_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_quantity,s_state,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_return_quantity] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_quarter_name,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_state] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_quantity,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_quantity] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_quarter_name,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_quarter_name] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_state] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity,cs_quantity] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_return_quantity] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_state,store_sales_quantitycount,s ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_quarter_name] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_quantity] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_quantity] + InputAdapter + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt index 516f782057..12e95ba50c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/explain.txt @@ -34,24 +34,24 @@ TakeOrderedAndProject (53) +- * Sort (46) +- Exchange (45) +- * Project (44) - +- * SortMergeJoin Inner (43) - :- * Sort (37) - : +- Exchange (36) - : +- * Project (35) - : +- * BroadcastHashJoin Inner BuildRight (34) - : :- * Project (29) - : : +- * Filter (28) - : : +- * ColumnarToRow (27) - : : +- Scan parquet default.customer (26) - : +- BroadcastExchange (33) - : +- * Filter (32) - : +- * ColumnarToRow (31) - : +- Scan parquet default.customer_address (30) - +- * Sort (42) - +- Exchange (41) - +- * Filter (40) - +- * ColumnarToRow (39) - +- Scan parquet default.customer_demographics (38) + +- * BroadcastHashJoin Inner BuildRight (43) + :- * Project (38) + : +- * SortMergeJoin Inner (37) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * Filter (28) + : : +- * ColumnarToRow (27) + : : +- Scan parquet default.customer (26) + : +- * Sort (36) + : +- Exchange (35) + : +- * Filter (34) + : +- * ColumnarToRow (33) + : +- Scan parquet default.customer_demographics (32) + +- BroadcastExchange (42) + +- * Filter (41) + +- * ColumnarToRow (40) + +- Scan parquet default.customer_address (39) (1) Scan parquet default.catalog_sales @@ -159,7 +159,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -172,89 +172,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [1,6,8,9,12,2]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [MS,IN,ND,OK,NM,VA]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -263,12 +263,12 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) Expand [codegen id : 13] -Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28] -Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, ca_state#29, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#30, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] +Input [11]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31] +Arguments: [List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 0), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, ca_state#32, null, 1), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, ca_country#33, null, null, 3), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#19, null, null, null, 7), List(cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, null, null, null, null, 15)], [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] (50) HashAggregate [codegen id : 13] Input [12]: [cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_birth_year#26, i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] @@ -279,7 +279,7 @@ Results [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_gro (51) Exchange Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] -Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), true, [id=#69] +Arguments: hashpartitioning(i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, 5), ENSURE_REQUIREMENTS, [id=#69] (52) HashAggregate [codegen id : 14] Input [19]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40, sum#55, count#56, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt index 8c76e7cab3..8069d43c34 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q18.sf100/simplified.txt @@ -54,34 +54,34 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (7) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt index 88b5168f60..4627bc19f2 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/explain.txt @@ -4,248 +4,248 @@ TakeOrderedAndProject (45) +- Exchange (43) +- * HashAggregate (42) +- * Project (41) - +- * SortMergeJoin Inner (40) - :- * Sort (25) - : +- Exchange (24) - : +- * Project (23) - : +- * BroadcastHashJoin Inner BuildRight (22) - : :- * Project (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) - : : :- * Project (10) - : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (8) - : : : +- * Project (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- BroadcastExchange (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.date_dim (11) - : +- BroadcastExchange (21) - : +- * Filter (20) - : +- * ColumnarToRow (19) - : +- Scan parquet default.store (18) - +- * Sort (39) - +- Exchange (38) - +- * Project (37) - +- * SortMergeJoin Inner (36) - :- * Sort (30) - : +- Exchange (29) - : +- * Filter (28) - : +- * ColumnarToRow (27) - : +- Scan parquet default.customer (26) - +- * Sort (35) - +- Exchange (34) - +- * Filter (33) - +- * ColumnarToRow (32) - +- Scan parquet default.customer_address (31) - - -(1) Scan parquet default.store_sales -Output [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] + +- * BroadcastHashJoin Inner BuildRight (40) + :- * Project (34) + : +- * SortMergeJoin Inner (33) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildLeft (9) + : : : :- BroadcastExchange (5) + : : : : +- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.date_dim (1) + : : : +- * Filter (8) + : : : +- * ColumnarToRow (7) + : : : +- Scan parquet default.store_sales (6) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (32) + : +- Exchange (31) + : +- * Project (30) + : +- * SortMergeJoin Inner (29) + : :- * Sort (23) + : : +- Exchange (22) + : : +- * Filter (21) + : : +- * ColumnarToRow (20) + : : +- Scan parquet default.customer (19) + : +- * Sort (28) + : +- Exchange (27) + : +- * Filter (26) + : +- * ColumnarToRow (25) + : +- Scan parquet default.customer_address (24) + +- BroadcastExchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.item (35) + + +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] -ReadSchema: struct - -(2) ColumnarToRow [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] - -(3) Filter [codegen id : 4] -Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5] -Condition : (((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) AND isnotnull(ss_customer_sk#3)) AND isnotnull(ss_store_sk#4)) - -(4) Scan parquet default.item -Output [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] -ReadSchema: struct - -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] - -(6) Filter [codegen id : 1] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] -Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 8)) AND isnotnull(i_item_sk#6)) +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [6]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, i_manager_id#11] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(8) BroadcastExchange -Input [5]: [i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1998)) AND isnotnull(d_date_sk#1)) -(9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#6] -Join condition: None +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(10) Project [codegen id : 4] -Output [8]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [10]: [ss_sold_date_sk#1, ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_item_sk#6, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#13, d_year#14, d_moy#15] +(6) Scan parquet default.store_sales +Output [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct - -(12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] - -(13) Filter [codegen id : 2] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -Condition : ((((isnotnull(d_moy#15) AND isnotnull(d_year#14)) AND (d_moy#15 = 11)) AND (d_year#14 = 1998)) AND isnotnull(d_date_sk#13)) +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk), IsNotNull(ss_customer_sk), IsNotNull(ss_store_sk)] +ReadSchema: struct -(14) Project [codegen id : 2] -Output [1]: [d_date_sk#13] -Input [3]: [d_date_sk#13, d_year#14, d_moy#15] +(7) ColumnarToRow +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(15) BroadcastExchange -Input [1]: [d_date_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(8) Filter +Input [5]: [ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Condition : (((isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) AND isnotnull(ss_customer_sk#7)) AND isnotnull(ss_store_sk#8)) -(16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#13] +(9) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None -(17) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_sold_date_sk#1, ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, d_date_sk#13] +(10) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] +Input [6]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9] -(18) Scan parquet default.store -Output [2]: [s_store_sk#17, s_zip#18] +(11) Scan parquet default.store +Output [2]: [s_store_sk#10, s_zip#11] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] +(12) ColumnarToRow [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] -(20) Filter [codegen id : 3] -Input [2]: [s_store_sk#17, s_zip#18] -Condition : (isnotnull(s_zip#18) AND isnotnull(s_store_sk#17)) +(13) Filter [codegen id : 2] +Input [2]: [s_store_sk#10, s_zip#11] +Condition : (isnotnull(s_zip#11) AND isnotnull(s_store_sk#10)) -(21) BroadcastExchange -Input [2]: [s_store_sk#17, s_zip#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(14) BroadcastExchange +Input [2]: [s_store_sk#10, s_zip#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] -(22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#4] -Right keys [1]: [s_store_sk#17] +(15) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [ss_store_sk#8] +Right keys [1]: [s_store_sk#10] Join condition: None -(23) Project [codegen id : 4] -Output [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Input [9]: [ss_customer_sk#3, ss_store_sk#4, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_store_sk#17, s_zip#18] +(16) Project [codegen id : 3] +Output [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_store_sk#8, ss_ext_sales_price#9, s_store_sk#10, s_zip#11] -(24) Exchange -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: hashpartitioning(ss_customer_sk#3, 5), true, [id=#20] +(17) Exchange +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: hashpartitioning(ss_customer_sk#7, 5), ENSURE_REQUIREMENTS, [id=#13] -(25) Sort [codegen id : 5] -Input [7]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18] -Arguments: [ss_customer_sk#3 ASC NULLS FIRST], false, 0 +(18) Sort [codegen id : 4] +Input [4]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11] +Arguments: [ss_customer_sk#7 ASC NULLS FIRST], false, 0 -(26) Scan parquet default.customer -Output [2]: [c_customer_sk#21, c_current_addr_sk#22] +(19) Scan parquet default.customer +Output [2]: [c_customer_sk#14, c_current_addr_sk#15] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +(20) ColumnarToRow [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] -(28) Filter [codegen id : 6] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Condition : (isnotnull(c_customer_sk#21) AND isnotnull(c_current_addr_sk#22)) +(21) Filter [codegen id : 5] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_current_addr_sk#15)) -(29) Exchange -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: hashpartitioning(c_current_addr_sk#22, 5), true, [id=#23] +(22) Exchange +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: hashpartitioning(c_current_addr_sk#15, 5), ENSURE_REQUIREMENTS, [id=#16] -(30) Sort [codegen id : 7] -Input [2]: [c_customer_sk#21, c_current_addr_sk#22] -Arguments: [c_current_addr_sk#22 ASC NULLS FIRST], false, 0 +(23) Sort [codegen id : 6] +Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +Arguments: [c_current_addr_sk#15 ASC NULLS FIRST], false, 0 -(31) Scan parquet default.customer_address -Output [2]: [ca_address_sk#24, ca_zip#25] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_zip#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(32) ColumnarToRow [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] -(33) Filter [codegen id : 8] -Input [2]: [ca_address_sk#24, ca_zip#25] -Condition : (isnotnull(ca_address_sk#24) AND isnotnull(ca_zip#25)) +(26) Filter [codegen id : 7] +Input [2]: [ca_address_sk#17, ca_zip#18] +Condition : (isnotnull(ca_address_sk#17) AND isnotnull(ca_zip#18)) -(34) Exchange -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: hashpartitioning(ca_address_sk#24, 5), true, [id=#26] +(27) Exchange +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] -(35) Sort [codegen id : 9] -Input [2]: [ca_address_sk#24, ca_zip#25] -Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#17, ca_zip#18] +Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 -(36) SortMergeJoin [codegen id : 10] -Left keys [1]: [c_current_addr_sk#22] -Right keys [1]: [ca_address_sk#24] +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [c_current_addr_sk#15] +Right keys [1]: [ca_address_sk#17] Join condition: None -(37) Project [codegen id : 10] -Output [2]: [c_customer_sk#21, ca_zip#25] -Input [4]: [c_customer_sk#21, c_current_addr_sk#22, ca_address_sk#24, ca_zip#25] +(30) Project [codegen id : 9] +Output [2]: [c_customer_sk#14, ca_zip#18] +Input [4]: [c_customer_sk#14, c_current_addr_sk#15, ca_address_sk#17, ca_zip#18] + +(31) Exchange +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#20] + +(32) Sort [codegen id : 10] +Input [2]: [c_customer_sk#14, ca_zip#18] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 -(38) Exchange -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: hashpartitioning(c_customer_sk#21, 5), true, [id=#27] +(33) SortMergeJoin [codegen id : 12] +Left keys [1]: [ss_customer_sk#7] +Right keys [1]: [c_customer_sk#14] +Join condition: NOT (substr(ca_zip#18, 1, 5) = substr(s_zip#11, 1, 5)) -(39) Sort [codegen id : 11] -Input [2]: [c_customer_sk#21, ca_zip#25] -Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 +(34) Project [codegen id : 12] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#9] +Input [6]: [ss_item_sk#6, ss_customer_sk#7, ss_ext_sales_price#9, s_zip#11, c_customer_sk#14, ca_zip#18] -(40) SortMergeJoin [codegen id : 12] -Left keys [1]: [ss_customer_sk#3] -Right keys [1]: [c_customer_sk#21] -Join condition: NOT (substr(ca_zip#25, 1, 5) = substr(s_zip#18, 1, 5)) +(35) Scan parquet default.item +Output [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,8), IsNotNull(i_item_sk)] +ReadSchema: struct + +(36) ColumnarToRow [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(37) Filter [codegen id : 11] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] +Condition : ((isnotnull(i_manager_id#26) AND (i_manager_id#26 = 8)) AND isnotnull(i_item_sk#21)) + +(38) Project [codegen id : 11] +Output [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [6]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25, i_manager_id#26] + +(39) BroadcastExchange +Input [5]: [i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] + +(40) BroadcastHashJoin [codegen id : 12] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#21] +Join condition: None (41) Project [codegen id : 12] -Output [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Input [9]: [ss_customer_sk#3, ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10, s_zip#18, c_customer_sk#21, ca_zip#25] +Output [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Input [7]: [ss_item_sk#6, ss_ext_sales_price#9, i_item_sk#21, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] (42) HashAggregate [codegen id : 12] -Input [5]: [ss_ext_sales_price#5, i_brand_id#7, i_brand#8, i_manufact_id#9, i_manufact#10] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#5))] +Input [5]: [ss_ext_sales_price#9, i_brand_id#22, i_brand#23, i_manufact_id#24, i_manufact#25] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#9))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] +Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] (43) Exchange -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Arguments: hashpartitioning(i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, 5), true, [id=#30] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Arguments: hashpartitioning(i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, 5), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -Input [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] -Keys [4]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#5))#31] -Results [5]: [i_brand_id#7 AS brand_id#32, i_brand#8 AS brand#33, i_manufact_id#9, i_manufact#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS ext_price#34] +Input [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Keys [4]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#9))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#9))#31] +Results [5]: [i_brand_id#22 AS brand_id#32, i_brand#23 AS brand#33, i_manufact_id#24, i_manufact#25, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#9))#31,17,2) AS ext_price#34] (45) TakeOrderedAndProject -Input [5]: [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] -Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#9 ASC NULLS FIRST, i_manufact#10 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#9, i_manufact#10, ext_price#34] +Input [5]: [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] +Arguments: 100, [ext_price#34 DESC NULLS LAST, brand#33 ASC NULLS FIRST, brand_id#32 ASC NULLS FIRST, i_manufact_id#24 ASC NULLS FIRST, i_manufact#25 ASC NULLS FIRST], [brand_id#32, brand#33, i_manufact_id#24, i_manufact#25, ext_price#34] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt index 05fa3f82e2..b6441c5fe7 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q19.sf100/simplified.txt @@ -6,71 +6,71 @@ TakeOrderedAndProject [ext_price,brand,brand_id,i_manufact_id,i_manufact] WholeStageCodegen (12) HashAggregate [i_brand,i_brand_id,i_manufact_id,i_manufact,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] - InputAdapter - WholeStageCodegen (5) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #2 - WholeStageCodegen (4) - Project [ss_customer_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact,s_zip] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price,i_brand_id,i_brand,i_manufact_id,i_manufact] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + SortMergeJoin [ss_customer_sk,c_customer_sk,ca_zip,s_zip] + InputAdapter + WholeStageCodegen (4) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #2 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ext_sales_price,s_zip] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Filter [s_zip,s_store_sk] - ColumnarToRow + Filter [s_zip,s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_zip] + InputAdapter + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #5 + WholeStageCodegen (9) + Project [c_customer_sk,ca_zip] + SortMergeJoin [c_current_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (6) + Sort [c_current_addr_sk] + InputAdapter + Exchange [c_current_addr_sk] #6 + WholeStageCodegen (5) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (8) + Sort [ca_address_sk] InputAdapter - Scan parquet default.store [s_store_sk,s_zip] + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) + Filter [ca_address_sk,ca_zip] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_zip] InputAdapter - WholeStageCodegen (11) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #6 - WholeStageCodegen (10) - Project [c_customer_sk,ca_zip] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (7) - Sort [c_current_addr_sk] - InputAdapter - Exchange [c_current_addr_sk] #7 - WholeStageCodegen (6) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (9) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #8 - WholeStageCodegen (8) - Filter [ca_address_sk,ca_zip] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_zip] + BroadcastExchange #8 + WholeStageCodegen (11) + Project [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact] + Filter [i_manager_id,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt index 61e5ae0121..52dfff442b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -* Sort (41) -+- Exchange (40) - +- * Project (39) - +- * SortMergeJoin Inner (38) - :- * Sort (26) - : +- * Project (25) - : +- * BroadcastHashJoin Inner BuildRight (24) - : :- * HashAggregate (18) - : : +- Exchange (17) - : : +- * HashAggregate (16) - : : +- * Project (15) - : : +- * BroadcastHashJoin Inner BuildRight (14) - : : :- Union (9) - : : : :- * Project (4) - : : : : +- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.web_sales (1) - : : : +- * Project (8) - : : : +- * Filter (7) - : : : +- * ColumnarToRow (6) - : : : +- Scan parquet default.catalog_sales (5) - : : +- BroadcastExchange (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) - : +- BroadcastExchange (23) - : +- * Project (22) - : +- * Filter (21) - : +- * ColumnarToRow (20) - : +- Scan parquet default.date_dim (19) - +- * Sort (37) - +- Exchange (36) - +- * Project (35) - +- * BroadcastHashJoin Inner BuildRight (34) - :- * HashAggregate (28) - : +- ReusedExchange (27) - +- BroadcastExchange (33) - +- * Project (32) - +- * Filter (31) - +- * ColumnarToRow (30) - +- Scan parquet default.date_dim (29) +* Sort (39) ++- Exchange (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * Project (25) + : +- * BroadcastHashJoin Inner BuildRight (24) + : :- * HashAggregate (18) + : : +- Exchange (17) + : : +- * HashAggregate (16) + : : +- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) + : : :- Union (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.web_sales (1) + : : : +- * Project (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.catalog_sales (5) + : : +- BroadcastExchange (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet default.date_dim (10) + : +- BroadcastExchange (23) + : +- * Project (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.date_dim (19) + +- BroadcastExchange (35) + +- * Project (34) + +- * BroadcastHashJoin Inner BuildRight (33) + :- * HashAggregate (27) + : +- ReusedExchange (26) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- * ColumnarToRow (29) + +- Scan parquet default.date_dim (28) (1) Scan parquet default.web_sales @@ -116,9 +114,9 @@ Results [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum (17) Exchange Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] -Arguments: hashpartitioning(d_week_seq#10, 5), true, [id=#27] +Arguments: hashpartitioning(d_week_seq#10, 5), ENSURE_REQUIREMENTS, [id=#27] -(18) HashAggregate [codegen id : 6] +(18) HashAggregate [codegen id : 12] Input [8]: [d_week_seq#10, sum#20, sum#21, sum#22, sum#23, sum#24, sum#25, sum#26] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))] @@ -147,82 +145,74 @@ Input [2]: [d_week_seq#42, d_year#43] Input [1]: [d_week_seq#42] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#44] -(24) BroadcastHashJoin [codegen id : 6] +(24) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#42] Join condition: None -(25) Project [codegen id : 6] +(25) Project [codegen id : 12] Output [8]: [d_week_seq#10 AS d_week_seq1#45, sun_sales#35 AS sun_sales1#46, mon_sales#36 AS mon_sales1#47, tue_sales#37 AS tue_sales1#48, wed_sales#38 AS wed_sales1#49, thu_sales#39 AS thu_sales1#50, fri_sales#40 AS fri_sales1#51, sat_sales#41 AS sat_sales1#52] Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#42] -(26) Sort [codegen id : 6] -Input [8]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52] -Arguments: [d_week_seq1#45 ASC NULLS FIRST], false, 0 - -(27) ReusedExchange [Reuses operator id: 17] +(26) ReusedExchange [Reuses operator id: 17] Output [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] -(28) HashAggregate [codegen id : 12] +(27) HashAggregate [codegen id : 11] Input [8]: [d_week_seq#10, sum#53, sum#54, sum#55, sum#56, sum#57, sum#58, sum#59] Keys [1]: [d_week_seq#10] Functions [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END)), sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))] Aggregate Attributes [7]: [sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#60, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#61, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#62, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#64, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#65, sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#66] Results [8]: [d_week_seq#10, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Sunday) THEN sales_price#4 ELSE null END))#60,17,2) AS sun_sales#35, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Monday) THEN sales_price#4 ELSE null END))#61,17,2) AS mon_sales#36, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Tuesday) THEN sales_price#4 ELSE null END))#62,17,2) AS tue_sales#37, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Wednesday) THEN sales_price#4 ELSE null END))#63,17,2) AS wed_sales#38, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Thursday) THEN sales_price#4 ELSE null END))#64,17,2) AS thu_sales#39, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Friday) THEN sales_price#4 ELSE null END))#65,17,2) AS fri_sales#40, MakeDecimal(sum(UnscaledValue(CASE WHEN (d_day_name#11 = Saturday) THEN sales_price#4 ELSE null END))#66,17,2) AS sat_sales#41] -(29) Scan parquet default.date_dim +(28) Scan parquet default.date_dim Output [2]: [d_week_seq#67, d_year#68] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_week_seq)] ReadSchema: struct -(30) ColumnarToRow [codegen id : 11] +(29) ColumnarToRow [codegen id : 10] Input [2]: [d_week_seq#67, d_year#68] -(31) Filter [codegen id : 11] +(30) Filter [codegen id : 10] Input [2]: [d_week_seq#67, d_year#68] Condition : ((isnotnull(d_year#68) AND (d_year#68 = 2002)) AND isnotnull(d_week_seq#67)) -(32) Project [codegen id : 11] +(31) Project [codegen id : 10] Output [1]: [d_week_seq#67] Input [2]: [d_week_seq#67, d_year#68] -(33) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_week_seq#67] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#69] -(34) BroadcastHashJoin [codegen id : 12] +(33) BroadcastHashJoin [codegen id : 11] Left keys [1]: [d_week_seq#10] Right keys [1]: [d_week_seq#67] Join condition: None -(35) Project [codegen id : 12] +(34) Project [codegen id : 11] Output [8]: [d_week_seq#10 AS d_week_seq2#70, sun_sales#35 AS sun_sales2#71, mon_sales#36 AS mon_sales2#72, tue_sales#37 AS tue_sales2#73, wed_sales#38 AS wed_sales2#74, thu_sales#39 AS thu_sales2#75, fri_sales#40 AS fri_sales2#76, sat_sales#41 AS sat_sales2#77] Input [9]: [d_week_seq#10, sun_sales#35, mon_sales#36, tue_sales#37, wed_sales#38, thu_sales#39, fri_sales#40, sat_sales#41, d_week_seq#67] -(36) Exchange -Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: hashpartitioning((d_week_seq2#70 - 53), 5), true, [id=#78] - -(37) Sort [codegen id : 13] +(35) BroadcastExchange Input [8]: [d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -Arguments: [(d_week_seq2#70 - 53) ASC NULLS FIRST], false, 0 +Arguments: HashedRelationBroadcastMode(List(cast((input[0, int, true] - 53) as bigint)),false), [id=#78] -(38) SortMergeJoin [codegen id : 14] +(36) BroadcastHashJoin [codegen id : 12] Left keys [1]: [d_week_seq1#45] Right keys [1]: [(d_week_seq2#70 - 53)] Join condition: None -(39) Project [codegen id : 14] +(37) Project [codegen id : 12] Output [8]: [d_week_seq1#45, round(CheckOverflow((promote_precision(sun_sales1#46) / promote_precision(sun_sales2#71)), DecimalType(37,20), true), 2) AS round((sun_sales1 / sun_sales2), 2)#79, round(CheckOverflow((promote_precision(mon_sales1#47) / promote_precision(mon_sales2#72)), DecimalType(37,20), true), 2) AS round((mon_sales1 / mon_sales2), 2)#80, round(CheckOverflow((promote_precision(tue_sales1#48) / promote_precision(tue_sales2#73)), DecimalType(37,20), true), 2) AS round((tue_sales1 / tue_sales2), 2)#81, round(CheckOverflow((promote_precision(wed_sales1#49) / promote_precision(wed_sales2#74)), DecimalType(37,20), true), 2) AS round((wed_sales1 / wed_sales2), 2)#82, round(CheckOverflow((promote_precision(thu_sales1#50) / promote_precision(thu_sales2#75)), DecimalType(37,20), true), 2) AS round((thu_sales1 / thu_sales2), 2)#83, round(CheckOverflow((promote_precision(fri_sales1#51) / promote_precision(fri_sales2#76)), DecimalType(37,20), true), 2) AS round((fri_sales1 / fri_sales2), 2)#84, round(CheckOverflow((promote_precision(sat_sales1#52) / promote_precision(sat_sales2#77)), DecimalType(37,20), true), 2) AS round((sat_sales1 / sat_sales2), 2)#85] Input [16]: [d_week_seq1#45, sun_sales1#46, mon_sales1#47, tue_sales1#48, wed_sales1#49, thu_sales1#50, fri_sales1#51, sat_sales1#52, d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77] -(40) Exchange +(38) Exchange Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] -Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), true, [id=#86] +Arguments: rangepartitioning(d_week_seq1#45 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#86] -(41) Sort [codegen id : 15] +(39) Sort [codegen id : 13] Input [8]: [d_week_seq1#45, round((sun_sales1 / sun_sales2), 2)#79, round((mon_sales1 / mon_sales2), 2)#80, round((tue_sales1 / tue_sales2), 2)#81, round((wed_sales1 / wed_sales2), 2)#82, round((thu_sales1 / thu_sales2), 2)#83, round((fri_sales1 / fri_sales2), 2)#84, round((sat_sales1 / sat_sales2), 2)#85] Arguments: [d_week_seq1#45 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt index 3389774c46..424a535e14 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/simplified.txt @@ -1,67 +1,61 @@ -WholeStageCodegen (15) +WholeStageCodegen (13) Sort [d_week_seq1] InputAdapter Exchange [d_week_seq1] #1 - WholeStageCodegen (14) + WholeStageCodegen (12) Project [d_week_seq1,sun_sales1,sun_sales2,mon_sales1,mon_sales2,tue_sales1,tue_sales2,wed_sales1,wed_sales2,thu_sales1,thu_sales2,fri_sales1,fri_sales2,sat_sales1,sat_sales2] - SortMergeJoin [d_week_seq1,d_week_seq2] + BroadcastHashJoin [d_week_seq1,d_week_seq2] + Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] + BroadcastHashJoin [d_week_seq,d_week_seq] + HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] + InputAdapter + Exchange [d_week_seq] #2 + WholeStageCodegen (4) + HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] + Project [sales_price,d_week_seq,d_day_name] + BroadcastHashJoin [sold_date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (1) + Project [ws_sold_date_sk,ws_ext_sales_price] + Filter [ws_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] + WholeStageCodegen (2) + Project [cs_sold_date_sk,cs_ext_sales_price] + Filter [cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (3) + Filter [d_date_sk,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + Project [d_week_seq] + Filter [d_year,d_week_seq] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_week_seq,d_year] InputAdapter - WholeStageCodegen (6) - Sort [d_week_seq1] + BroadcastExchange #5 + WholeStageCodegen (11) Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] BroadcastHashJoin [d_week_seq,d_week_seq] HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] InputAdapter - Exchange [d_week_seq] #2 - WholeStageCodegen (4) - HashAggregate [d_week_seq,d_day_name,sales_price] [sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum,sum] - Project [sales_price,d_week_seq,d_day_name] - BroadcastHashJoin [sold_date_sk,d_date_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [ws_sold_date_sk,ws_ext_sales_price] - Filter [ws_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_ext_sales_price] - WholeStageCodegen (2) - Project [cs_sold_date_sk,cs_ext_sales_price] - Filter [cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (3) - Filter [d_date_sk,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq,d_day_name] + ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) + BroadcastExchange #6 + WholeStageCodegen (10) Project [d_week_seq] Filter [d_year,d_week_seq] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_week_seq,d_year] - InputAdapter - WholeStageCodegen (13) - Sort [d_week_seq2] - InputAdapter - Exchange [d_week_seq2] #5 - WholeStageCodegen (12) - Project [d_week_seq,sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales] - BroadcastHashJoin [d_week_seq,d_week_seq] - HashAggregate [d_week_seq,sum,sum,sum,sum,sum,sum,sum] [sum(UnscaledValue(CASE WHEN (d_day_name = Sunday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Monday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Tuesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Wednesday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Thursday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Friday) THEN sales_price ELSE null END)),sum(UnscaledValue(CASE WHEN (d_day_name = Saturday) THEN sales_price ELSE null END)),sun_sales,mon_sales,tue_sales,wed_sales,thu_sales,fri_sales,sat_sales,sum,sum,sum,sum,sum,sum,sum] - InputAdapter - ReusedExchange [d_week_seq,sum,sum,sum,sum,sum,sum,sum] #2 - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (11) - Project [d_week_seq] - Filter [d_year,d_week_seq] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_week_seq,d_year] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt index ffcf6bd4f6..093c4eed6c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt index 10f874f8f5..7de562c5d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt index 73f36e3a9c..273950bed3 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt @@ -92,7 +92,7 @@ Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, (10) Exchange Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#13] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#13] (11) Sort [codegen id : 3] Input [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] @@ -114,7 +114,7 @@ Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (15) Exchange Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#18] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#18] (16) Sort [codegen id : 5] Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] @@ -189,7 +189,7 @@ Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_c (32) Exchange Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#29] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#29] (33) Sort [codegen id : 9] Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, s_store_name#20, s_state#22, ca_state#25] @@ -211,7 +211,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (37) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#32] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] (38) Sort [codegen id : 11] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -235,7 +235,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (42) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#35] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#35] (43) HashAggregate [codegen id : 13] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#34] @@ -253,7 +253,7 @@ Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty# (45) Exchange Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), true, [id=#42] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, 5), ENSURE_REQUIREMENTS, [id=#42] (46) HashAggregate [codegen id : 14] Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] @@ -309,12 +309,12 @@ Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquer : : : +- Exchange (64) : : : +- * Filter (63) : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.item (61) + : : : +- Scan parquet default.customer (61) : : +- * Sort (74) : : +- Exchange (73) : : +- * Filter (72) : : +- * ColumnarToRow (71) - : : +- Scan parquet default.customer (70) + : : +- Scan parquet default.item (70) : +- * Sort (83) : +- Exchange (82) : +- * Filter (81) @@ -374,88 +374,88 @@ Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, s (59) Exchange Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: hashpartitioning(ss_item_sk#1, 5), true, [id=#49] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] (60) Sort [codegen id : 3] Input [7]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23] -Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 +Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.item -Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(61) Scan parquet default.customer +Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] +ReadSchema: struct (62) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (63) Filter [codegen id : 4] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Condition : isnotnull(i_item_sk#6) +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) (64) Exchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(i_item_sk#6, 5), true, [id=#50] +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(c_customer_sk#14, 5), ENSURE_REQUIREMENTS, [id=#50] (65) Sort [codegen id : 5] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 +Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 (66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Left keys [1]: [ss_customer_sk#2] +Right keys [1]: [c_customer_sk#14] Join condition: None (67) Project [codegen id : 6] -Output [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Input [13]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Output [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] (68) Exchange -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: hashpartitioning(ss_customer_sk#2, 5), true, [id=#51] +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [id=#51] (69) Sort [codegen id : 7] -Input [12]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [9]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17] +Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0 -(70) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(70) Scan parquet default.item +Output [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_item_sk)] +ReadSchema: struct (71) ColumnarToRow [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (72) Filter [codegen id : 8] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Condition : (isnotnull(c_customer_sk#14) AND isnotnull(c_birth_country#17)) +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Condition : isnotnull(i_item_sk#6) (73) Exchange -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_customer_sk#14, 5), true, [id=#52] +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(i_item_sk#6, 5), ENSURE_REQUIREMENTS, [id=#52] (74) Sort [codegen id : 9] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: [c_customer_sk#14 ASC NULLS FIRST], false, 0 +Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: [i_item_sk#6 ASC NULLS FIRST], false, 0 (75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#6] Join condition: None (76) Project [codegen id : 10] -Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [16]: [ss_item_sk#1, ss_customer_sk#2, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +Output [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] (77) Exchange -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] -Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), true, [id=#53] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +Arguments: hashpartitioning(c_birth_country#17, s_zip#23, 5), ENSURE_REQUIREMENTS, [id=#53] (78) Sort [codegen id : 11] -Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17] +Input [14]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] Arguments: [c_birth_country#17 ASC NULLS FIRST, s_zip#23 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer_address @@ -474,7 +474,7 @@ Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) (82) Exchange Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), true, [id=#54] +Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] (83) Sort [codegen id : 13] Input [3]: [ca_state#25, ca_zip#26, ca_country#27] @@ -486,15 +486,15 @@ Right keys [2]: [upper(ca_country#27), ca_zip#26] Join condition: None (85) Project [codegen id : 14] -Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, c_birth_country#17, ca_state#25, ca_zip#26, ca_country#27] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Input [17]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, s_zip#23, c_first_name#15, c_last_name#16, c_birth_country#17, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, ca_zip#26, ca_country#27] (86) Exchange -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), true, [id=#55] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#55] (87) Sort [codegen id : 15] -Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 (88) Scan parquet default.store_returns @@ -513,7 +513,7 @@ Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) (91) Exchange Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), true, [id=#56] +Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] (92) Sort [codegen id : 17] Input [2]: [sr_item_sk#30, sr_ticket_number#31] @@ -526,7 +526,7 @@ Join condition: None (94) Project [codegen id : 18] Output [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] -Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25, sr_item_sk#30, sr_ticket_number#31] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, s_store_name#20, s_state#22, c_first_name#15, c_last_name#16, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, ca_state#25, sr_item_sk#30, sr_ticket_number#31] (95) HashAggregate [codegen id : 18] Input [11]: [ss_net_paid#5, s_store_name#20, s_state#22, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11, c_first_name#15, c_last_name#16, ca_state#25] @@ -537,7 +537,7 @@ Results [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_ (96) Exchange Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] -Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), true, [id=#59] +Arguments: hashpartitioning(c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, 5), ENSURE_REQUIREMENTS, [id=#59] (97) HashAggregate [codegen id : 19] Input [11]: [c_last_name#16, c_first_name#15, s_store_name#20, ca_state#25, s_state#22, i_color#9, i_current_price#7, i_manager_id#11, i_units#10, i_size#8, sum#58] @@ -555,7 +555,7 @@ Results [2]: [sum#63, count#64] (99) Exchange Input [2]: [sum#63, count#64] -Arguments: SinglePartition, true, [id=#65] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] (100) HashAggregate [codegen id : 20] Input [2]: [sum#63, count#64] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt index 10f874f8f5..7de562c5d5 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/simplified.txt @@ -21,7 +21,7 @@ WholeStageCodegen (14) InputAdapter Exchange [ss_ticket_number,ss_item_sk] #12 WholeStageCodegen (14) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,ca_state] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] SortMergeJoin [c_birth_country,s_zip,ca_country,ca_zip] InputAdapter WholeStageCodegen (11) @@ -29,21 +29,21 @@ WholeStageCodegen (14) InputAdapter Exchange [c_birth_country,s_zip] #13 WholeStageCodegen (10) - Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id,c_first_name,c_last_name,c_birth_country] - SortMergeJoin [ss_customer_sk,c_customer_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country,i_current_price,i_size,i_color,i_units,i_manager_id] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter WholeStageCodegen (7) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #14 + Exchange [ss_item_sk] #14 WholeStageCodegen (6) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,i_current_price,i_size,i_color,i_units,i_manager_id] - SortMergeJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip,c_first_name,c_last_name,c_birth_country] + SortMergeJoin [ss_customer_sk,c_customer_sk] InputAdapter WholeStageCodegen (3) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #15 + Exchange [ss_customer_sk] #15 WholeStageCodegen (2) Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_paid,s_store_name,s_state,s_zip] BroadcastHashJoin [ss_store_sk,s_store_sk] @@ -61,24 +61,24 @@ WholeStageCodegen (14) Scan parquet default.store [s_store_sk,s_store_name,s_market_id,s_state,s_zip] InputAdapter WholeStageCodegen (5) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #17 + Exchange [c_customer_sk] #17 WholeStageCodegen (4) - Filter [i_item_sk] + Filter [c_customer_sk,c_birth_country] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] + Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] InputAdapter WholeStageCodegen (9) - Sort [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #18 + Exchange [i_item_sk] #18 WholeStageCodegen (8) - Filter [c_customer_sk,c_birth_country] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer [c_customer_sk,c_first_name,c_last_name,c_birth_country] + Scan parquet default.item [i_item_sk,i_current_price,i_size,i_color,i_units,i_manager_id] InputAdapter WholeStageCodegen (13) Sort [ca_country,ca_zip] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt index c6dc3db869..3100e574e6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/explain.txt @@ -5,57 +5,57 @@ TakeOrderedAndProject (57) +- * HashAggregate (54) +- * Project (53) +- * SortMergeJoin Inner (52) - :- * Sort (43) - : +- Exchange (42) - : +- * Project (41) - : +- * SortMergeJoin Inner (40) - : :- * Sort (27) - : : +- Exchange (26) - : : +- * Project (25) - : : +- * SortMergeJoin Inner (24) - : : :- * Sort (18) - : : : +- Exchange (17) - : : : +- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (10) - : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.store_sales (1) - : : : : +- BroadcastExchange (8) - : : : : +- * Project (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.date_dim (4) - : : : +- BroadcastExchange (14) - : : : +- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.store (11) - : : +- * Sort (23) - : : +- Exchange (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.item (19) - : +- * Sort (39) - : +- Exchange (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * Filter (30) - : : +- * ColumnarToRow (29) - : : +- Scan parquet default.store_returns (28) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.date_dim (31) + :- * Sort (27) + : +- Exchange (26) + : +- * Project (25) + : +- * SortMergeJoin Inner (24) + : :- * Sort (18) + : : +- Exchange (17) + : : +- * Project (16) + : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- BroadcastExchange (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.store (11) + : +- * Sort (23) + : +- Exchange (22) + : +- * Filter (21) + : +- * ColumnarToRow (20) + : +- Scan parquet default.item (19) +- * Sort (51) +- Exchange (50) +- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Filter (46) - : +- * ColumnarToRow (45) - : +- Scan parquet default.catalog_sales (44) - +- ReusedExchange (47) + +- * SortMergeJoin Inner (48) + :- * Sort (39) + : +- Exchange (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Filter (30) + : : +- * ColumnarToRow (29) + : : +- Scan parquet default.store_returns (28) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.date_dim (31) + +- * Sort (47) + +- Exchange (46) + +- * Project (45) + +- * BroadcastHashJoin Inner BuildRight (44) + :- * Filter (42) + : +- * ColumnarToRow (41) + : +- Scan parquet default.catalog_sales (40) + +- ReusedExchange (43) (1) Scan parquet default.store_sales @@ -132,7 +132,7 @@ Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_store_sk#4, ss_ticket_number#5, s (17) Exchange Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#15] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#15] (18) Sort [codegen id : 4] Input [6]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13] @@ -154,7 +154,7 @@ Condition : isnotnull(i_item_sk#16) (22) Exchange Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(i_item_sk#16, 5), true, [id=#19] +Arguments: hashpartitioning(i_item_sk#16, 5), ENSURE_REQUIREMENTS, [id=#19] (23) Sort [codegen id : 6] Input [3]: [i_item_sk#16, i_item_id#17, i_item_desc#18] @@ -171,7 +171,7 @@ Input [9]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, (26) Exchange Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), true, [id=#20] +Arguments: hashpartitioning(cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint), 5), ENSURE_REQUIREMENTS, [id=#20] (27) Sort [codegen id : 8] Input [8]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] @@ -224,89 +224,89 @@ Input [6]: [sr_returned_date_sk#21, sr_item_sk#22, sr_customer_sk#23, sr_ticket_ (38) Exchange Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), true, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), ENSURE_REQUIREMENTS, [id=#30] (39) Sort [codegen id : 11] Input [4]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] -Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 - -(40) SortMergeJoin [codegen id : 12] -Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] -Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] -Join condition: None - -(41) Project [codegen id : 12] -Output [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Input [12]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25] - -(42) Exchange -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] -Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, 5), true, [id=#31] - -(43) Sort [codegen id : 13] -Input [8]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25] Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST], false, 0 -(44) Scan parquet default.catalog_sales -Output [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(40) Scan parquet default.catalog_sales +Output [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(45) ColumnarToRow [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +(41) ColumnarToRow [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] + +(42) Filter [codegen id : 13] +Input [4]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Condition : ((isnotnull(cs_bill_customer_sk#32) AND isnotnull(cs_item_sk#33)) AND isnotnull(cs_sold_date_sk#31)) -(46) Filter [codegen id : 15] -Input [4]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Condition : ((isnotnull(cs_bill_customer_sk#33) AND isnotnull(cs_item_sk#34)) AND isnotnull(cs_sold_date_sk#32)) +(43) ReusedExchange [Reuses operator id: 35] +Output [1]: [d_date_sk#35] -(47) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#36] +(44) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cs_sold_date_sk#31] +Right keys [1]: [d_date_sk#35] +Join condition: None + +(45) Project [codegen id : 13] +Output [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Input [5]: [cs_sold_date_sk#31, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34, d_date_sk#35] + +(46) Exchange +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: hashpartitioning(cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint), 5), ENSURE_REQUIREMENTS, [id=#36] -(48) BroadcastHashJoin [codegen id : 15] -Left keys [1]: [cs_sold_date_sk#32] -Right keys [1]: [d_date_sk#36] +(47) Sort [codegen id : 14] +Input [3]: [cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] +Arguments: [cast(cs_bill_customer_sk#32 as bigint) ASC NULLS FIRST, cast(cs_item_sk#33 as bigint) ASC NULLS FIRST], false, 0 + +(48) SortMergeJoin [codegen id : 15] +Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] +Right keys [2]: [cast(cs_bill_customer_sk#32 as bigint), cast(cs_item_sk#33 as bigint)] Join condition: None (49) Project [codegen id : 15] -Output [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Input [5]: [cs_sold_date_sk#32, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35, d_date_sk#36] +Output [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Input [7]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_bill_customer_sk#32, cs_item_sk#33, cs_net_profit#34] (50) Exchange -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: hashpartitioning(cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint), 5), true, [id=#37] +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 5), ENSURE_REQUIREMENTS, [id=#37] (51) Sort [codegen id : 16] -Input [3]: [cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] -Arguments: [cast(cs_bill_customer_sk#33 as bigint) ASC NULLS FIRST, cast(cs_item_sk#34 as bigint) ASC NULLS FIRST], false, 0 +Input [5]: [sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] +Arguments: [sr_customer_sk#23 ASC NULLS FIRST, sr_item_sk#22 ASC NULLS FIRST, sr_ticket_number#24 ASC NULLS FIRST], false, 0 (52) SortMergeJoin [codegen id : 17] -Left keys [2]: [sr_customer_sk#23, sr_item_sk#22] -Right keys [2]: [cast(cs_bill_customer_sk#33 as bigint), cast(cs_item_sk#34 as bigint)] +Left keys [3]: [cast(ss_customer_sk#3 as bigint), cast(ss_item_sk#2 as bigint), cast(ss_ticket_number#5 as bigint)] +Right keys [3]: [sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24] Join condition: None (53) Project [codegen id : 17] -Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] -Input [11]: [ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_net_loss#25, cs_bill_customer_sk#33, cs_item_sk#34, cs_net_profit#35] +Output [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [13]: [ss_item_sk#2, ss_customer_sk#3, ss_ticket_number#5, ss_net_profit#6, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18, sr_item_sk#22, sr_customer_sk#23, sr_ticket_number#24, sr_net_loss#25, cs_net_profit#34] (54) HashAggregate [codegen id : 17] -Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#35, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] +Input [7]: [ss_net_profit#6, sr_net_loss#25, cs_net_profit#34, s_store_id#12, s_store_name#13, i_item_id#17, i_item_desc#18] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#34))] Aggregate Attributes [3]: [sum#38, sum#39, sum#40] Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] (55) Exchange Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] -Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), true, [id=#44] +Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, 5), ENSURE_REQUIREMENTS, [id=#44] (56) HashAggregate [codegen id : 18] Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, sum#41, sum#42, sum#43] Keys [4]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13] -Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#35))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#35))#47] -Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#35))#47,17,2) AS catalog_sales_profit#50] +Functions [3]: [sum(UnscaledValue(ss_net_profit#6)), sum(UnscaledValue(sr_net_loss#25)), sum(UnscaledValue(cs_net_profit#34))] +Aggregate Attributes [3]: [sum(UnscaledValue(ss_net_profit#6))#45, sum(UnscaledValue(sr_net_loss#25))#46, sum(UnscaledValue(cs_net_profit#34))#47] +Results [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, MakeDecimal(sum(UnscaledValue(ss_net_profit#6))#45,17,2) AS store_sales_profit#48, MakeDecimal(sum(UnscaledValue(sr_net_loss#25))#46,17,2) AS store_returns_loss#49, MakeDecimal(sum(UnscaledValue(cs_net_profit#34))#47,17,2) AS catalog_sales_profit#50] (57) TakeOrderedAndProject Input [7]: [i_item_id#17, i_item_desc#18, s_store_id#12, s_store_name#13, store_sales_profit#48, store_returns_loss#49, catalog_sales_profit#50] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt index ad9fa718ff..9b53cdaa5d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q25.sf100/simplified.txt @@ -6,67 +6,67 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales WholeStageCodegen (17) HashAggregate [i_item_id,i_item_desc,s_store_id,s_store_name,ss_net_profit,sr_net_loss,cs_net_profit] [sum,sum,sum,sum,sum,sum] Project [ss_net_profit,sr_net_loss,cs_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] + SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - WholeStageCodegen (13) - Sort [sr_customer_sk,sr_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #2 - WholeStageCodegen (12) - Project [ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc,sr_item_sk,sr_customer_sk,sr_net_loss] - SortMergeJoin [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #2 + WholeStageCodegen (7) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] + SortMergeJoin [ss_item_sk,i_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (4) + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk,ss_item_sk,ss_ticket_number] #3 - WholeStageCodegen (7) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name,i_item_id,i_item_desc] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (3) - Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] - BroadcastHashJoin [ss_store_sk,s_store_sk] - Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id,s_store_name] - InputAdapter - WholeStageCodegen (6) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #7 - WholeStageCodegen (5) - Filter [i_item_sk] + Exchange [ss_item_sk] #3 + WholeStageCodegen (3) + Project [ss_item_sk,ss_customer_sk,ss_ticket_number,ss_net_profit,s_store_id,s_store_name] + BroadcastHashJoin [ss_store_sk,s_store_sk] + Project [ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_item_sk,ss_ticket_number,ss_sold_date_sk,ss_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_customer_sk,ss_store_sk,ss_ticket_number,ss_net_profit] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (2) + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id,s_store_name] + InputAdapter + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #6 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] + InputAdapter + WholeStageCodegen (16) + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + InputAdapter + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #7 + WholeStageCodegen (15) + Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss,cs_net_profit] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] + Sort [sr_customer_sk,sr_item_sk] InputAdapter - Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #8 + Exchange [sr_customer_sk,sr_item_sk] #8 WholeStageCodegen (10) Project [sr_item_sk,sr_customer_sk,sr_ticket_number,sr_net_loss] BroadcastHashJoin [sr_returned_date_sk,d_date_sk] @@ -82,17 +82,17 @@ TakeOrderedAndProject [i_item_id,i_item_desc,s_store_id,s_store_name,store_sales ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - WholeStageCodegen (16) - Sort [cs_bill_customer_sk,cs_item_sk] - InputAdapter - Exchange [cs_bill_customer_sk,cs_item_sk] #10 - WholeStageCodegen (15) - Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] - Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] InputAdapter - ReusedExchange [d_date_sk] #9 + WholeStageCodegen (14) + Sort [cs_bill_customer_sk,cs_item_sk] + InputAdapter + Exchange [cs_bill_customer_sk,cs_item_sk] #10 + WholeStageCodegen (13) + Project [cs_bill_customer_sk,cs_item_sk,cs_net_profit] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Filter [cs_bill_customer_sk,cs_item_sk,cs_sold_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk,cs_net_profit] + InputAdapter + ReusedExchange [d_date_sk] #9 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt index 8185680b58..cb8522545f 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/explain.txt @@ -9,8 +9,8 @@ TakeOrderedAndProject (67) : +- * HashAggregate (30) : +- * Project (29) : +- * BroadcastHashJoin Inner BuildRight (28) - : :- * Project (22) - : : +- * BroadcastHashJoin Inner BuildRight (21) + : :- * Project (17) + : : +- * BroadcastHashJoin Inner BuildRight (16) : : :- * Project (10) : : : +- * BroadcastHashJoin Inner BuildRight (9) : : : :- * Filter (3) @@ -21,21 +21,21 @@ TakeOrderedAndProject (67) : : : +- * Filter (6) : : : +- * ColumnarToRow (5) : : : +- Scan parquet default.date_dim (4) - : : +- BroadcastExchange (20) - : : +- * BroadcastHashJoin LeftSemi BuildRight (19) - : : :- * Filter (13) - : : : +- * ColumnarToRow (12) - : : : +- Scan parquet default.item (11) - : : +- BroadcastExchange (18) - : : +- * Project (17) - : : +- * Filter (16) - : : +- * ColumnarToRow (15) - : : +- Scan parquet default.item (14) + : : +- BroadcastExchange (15) + : : +- * Project (14) + : : +- * Filter (13) + : : +- * ColumnarToRow (12) + : : +- Scan parquet default.customer_address (11) : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.customer_address (23) + : +- * BroadcastHashJoin LeftSemi BuildRight (26) + : :- * Filter (20) + : : +- * ColumnarToRow (19) + : : +- Scan parquet default.item (18) + : +- BroadcastExchange (25) + : +- * Project (24) + : +- * Filter (23) + : +- * ColumnarToRow (22) + : +- Scan parquet default.item (21) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -113,108 +113,108 @@ Join condition: None Output [3]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] Input [5]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, d_date_sk#5] -(11) Scan parquet default.item -Output [2]: [i_item_sk#9, i_manufact_id#10] +(11) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_gmt_offset#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(12) ColumnarToRow [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(13) Filter [codegen id : 2] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] +Condition : ((isnotnull(ca_gmt_offset#10) AND (ca_gmt_offset#10 = -5.00)) AND isnotnull(ca_address_sk#9)) + +(14) Project [codegen id : 2] +Output [1]: [ca_address_sk#9] +Input [2]: [ca_address_sk#9, ca_gmt_offset#10] + +(15) BroadcastExchange +Input [1]: [ca_address_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] + +(16) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None + +(17) Project [codegen id : 5] +Output [2]: [ss_item_sk#2, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, ca_address_sk#9] + +(18) Scan parquet default.item +Output [2]: [i_item_sk#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] +(19) ColumnarToRow [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] -(13) Filter [codegen id : 3] -Input [2]: [i_item_sk#9, i_manufact_id#10] -Condition : isnotnull(i_item_sk#9) +(20) Filter [codegen id : 4] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Condition : isnotnull(i_item_sk#12) -(14) Scan parquet default.item -Output [2]: [i_category#11, i_manufact_id#10] +(21) Scan parquet default.item +Output [2]: [i_category#14, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] ReadSchema: struct -(15) ColumnarToRow [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] +(22) ColumnarToRow [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] -(16) Filter [codegen id : 2] -Input [2]: [i_category#11, i_manufact_id#10] -Condition : (isnotnull(i_category#11) AND (i_category#11 = Electronics)) +(23) Filter [codegen id : 3] +Input [2]: [i_category#14, i_manufact_id#13] +Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) -(17) Project [codegen id : 2] -Output [1]: [i_manufact_id#10 AS i_manufact_id#10#12] -Input [2]: [i_category#11, i_manufact_id#10] +(24) Project [codegen id : 3] +Output [1]: [i_manufact_id#13 AS i_manufact_id#13#15] +Input [2]: [i_category#14, i_manufact_id#13] -(18) BroadcastExchange -Input [1]: [i_manufact_id#10#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#13] +(25) BroadcastExchange +Input [1]: [i_manufact_id#13#15] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(19) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [i_manufact_id#10] -Right keys [1]: [i_manufact_id#10#12] +(26) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [i_manufact_id#13] +Right keys [1]: [i_manufact_id#13#15] Join condition: None -(20) BroadcastExchange -Input [2]: [i_item_sk#9, i_manufact_id#10] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#14] - -(21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#9] -Join condition: None - -(22) Project [codegen id : 5] -Output [3]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10] -Input [5]: [ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4, i_item_sk#9, i_manufact_id#10] - -(23) Scan parquet default.customer_address -Output [2]: [ca_address_sk#15, ca_gmt_offset#16] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-5.00), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(24) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - -(25) Filter [codegen id : 4] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] -Condition : ((isnotnull(ca_gmt_offset#16) AND (ca_gmt_offset#16 = -5.00)) AND isnotnull(ca_address_sk#15)) - -(26) Project [codegen id : 4] -Output [1]: [ca_address_sk#15] -Input [2]: [ca_address_sk#15, ca_gmt_offset#16] - (27) BroadcastExchange -Input [1]: [ca_address_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [2]: [i_item_sk#12, i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_addr_sk#3] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#12] Join condition: None (29) Project [codegen id : 5] -Output [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Input [4]: [ss_addr_sk#3, ss_ext_sales_price#4, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Input [4]: [ss_item_sk#2, ss_ext_sales_price#4, i_item_sk#12, i_manufact_id#13] (30) HashAggregate [codegen id : 5] -Input [2]: [ss_ext_sales_price#4, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ss_ext_sales_price#4, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum#18] -Results [2]: [i_manufact_id#10, sum#19] +Results [2]: [i_manufact_id#13, sum#19] (31) Exchange -Input [2]: [i_manufact_id#10, sum#19] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#20] +Input [2]: [i_manufact_id#13, sum#19] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] (32) HashAggregate [codegen id : 6] -Input [2]: [i_manufact_id#10, sum#19] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#19] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#4))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#4))#21] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] (33) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] @@ -242,47 +242,47 @@ Join condition: None Output [3]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] Input [5]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, d_date_sk#5] -(39) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(39) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#9] Join condition: None (41) Project [codegen id : 11] -Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10] -Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#9, i_manufact_id#10] +Output [2]: [cs_item_sk#25, cs_ext_sales_price#26] +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, ca_address_sk#9] (42) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#12] Join condition: None (44) Project [codegen id : 11] -Output [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#10, ca_address_sk#15] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Input [4]: [cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#12, i_manufact_id#13] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#10, sum#28] +Results [2]: [i_manufact_id#13, sum#28] (46) Exchange -Input [2]: [i_manufact_id#10, sum#28] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#29] +Input [2]: [i_manufact_id#13, sum#28] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#10, sum#28] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#28] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#26))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#30] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] (48) Scan parquet default.web_sales Output [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] @@ -310,69 +310,69 @@ Join condition: None Output [3]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] Input [5]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, d_date_sk#5] -(54) ReusedExchange [Reuses operator id: 20] -Output [2]: [i_item_sk#9, i_manufact_id#10] +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [ca_address_sk#9] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#9] +Left keys [1]: [ws_bill_addr_sk#34] +Right keys [1]: [ca_address_sk#9] Join condition: None (56) Project [codegen id : 17] -Output [3]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10] -Input [5]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, i_item_sk#9, i_manufact_id#10] +Output [2]: [ws_item_sk#33, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35, ca_address_sk#9] (57) ReusedExchange [Reuses operator id: 27] -Output [1]: [ca_address_sk#15] +Output [2]: [i_item_sk#12, i_manufact_id#13] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#15] +Left keys [1]: [ws_item_sk#33] +Right keys [1]: [i_item_sk#12] Join condition: None (59) Project [codegen id : 17] -Output [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Input [4]: [ws_bill_addr_sk#34, ws_ext_sales_price#35, i_manufact_id#10, ca_address_sk#15] +Output [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Input [4]: [ws_item_sk#33, ws_ext_sales_price#35, i_item_sk#12, i_manufact_id#13] (60) HashAggregate [codegen id : 17] -Input [2]: [ws_ext_sales_price#35, i_manufact_id#10] -Keys [1]: [i_manufact_id#10] +Input [2]: [ws_ext_sales_price#35, i_manufact_id#13] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum#36] -Results [2]: [i_manufact_id#10, sum#37] +Results [2]: [i_manufact_id#13, sum#37] (61) Exchange -Input [2]: [i_manufact_id#10, sum#37] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#38] +Input [2]: [i_manufact_id#13, sum#37] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] (62) HashAggregate [codegen id : 18] -Input [2]: [i_manufact_id#10, sum#37] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, sum#37] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#35))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#35))#39] -Results [2]: [i_manufact_id#10, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +Results [2]: [i_manufact_id#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] (63) Union (64) HashAggregate [codegen id : 19] -Input [2]: [i_manufact_id#10, total_sales#22] -Keys [1]: [i_manufact_id#10] +Input [2]: [i_manufact_id#13, total_sales#22] +Keys [1]: [i_manufact_id#13] Functions [1]: [partial_sum(total_sales#22)] Aggregate Attributes [2]: [sum#41, isEmpty#42] -Results [3]: [i_manufact_id#10, sum#43, isEmpty#44] +Results [3]: [i_manufact_id#13, sum#43, isEmpty#44] (65) Exchange -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#10, 5), true, [id=#45] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] (66) HashAggregate [codegen id : 20] -Input [3]: [i_manufact_id#10, sum#43, isEmpty#44] -Keys [1]: [i_manufact_id#10] +Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] +Keys [1]: [i_manufact_id#13] Functions [1]: [sum(total_sales#22)] Aggregate Attributes [1]: [sum(total_sales#22)#46] -Results [2]: [i_manufact_id#10, sum(total_sales#22)#46 AS total_sales#47] +Results [2]: [i_manufact_id#13, sum(total_sales#22)#46 AS total_sales#47] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#10, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#10, total_sales#47] +Input [2]: [i_manufact_id#13, total_sales#47] +Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt index 410def2466..14787f0bbc 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q33.sf100/simplified.txt @@ -14,9 +14,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (5) HashAggregate [i_manufact_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] - Project [ss_addr_sk,ss_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ss_item_sk,i_item_sk] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_addr_sk,ca_address_sk] Project [ss_item_sk,ss_addr_sk,ss_ext_sales_price] BroadcastHashJoin [ss_sold_date_sk,d_date_sk] Filter [ss_sold_date_sk,ss_addr_sk,ss_item_sk] @@ -33,28 +33,28 @@ TakeOrderedAndProject [total_sales,i_manufact_id] Scan parquet default.date_dim [d_date_sk,d_year,d_moy] InputAdapter BroadcastExchange #4 - WholeStageCodegen (3) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + WholeStageCodegen (2) + Project [ca_address_sk] + Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_manufact_id] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (2) - Project [i_manufact_id] - Filter [i_category] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_category,i_manufact_id] + Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #5 WholeStageCodegen (4) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + BroadcastHashJoin [i_manufact_id,i_manufact_id] + Filter [i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] + Scan parquet default.item [i_item_sk,i_manufact_id] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] WholeStageCodegen (12) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(cs_ext_sales_price)),total_sales,sum] InputAdapter @@ -62,9 +62,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (11) HashAggregate [i_manufact_id,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] - Project [cs_bill_addr_sk,cs_ext_sales_price,i_manufact_id] - BroadcastHashJoin [cs_item_sk,i_item_sk] + BroadcastHashJoin [cs_item_sk,i_item_sk] + Project [cs_item_sk,cs_ext_sales_price] + BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] Project [cs_bill_addr_sk,cs_item_sk,cs_ext_sales_price] BroadcastHashJoin [cs_sold_date_sk,d_date_sk] Filter [cs_sold_date_sk,cs_bill_addr_sk,cs_item_sk] @@ -74,9 +74,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 WholeStageCodegen (18) HashAggregate [i_manufact_id,sum] [sum(UnscaledValue(ws_ext_sales_price)),total_sales,sum] InputAdapter @@ -84,9 +84,9 @@ TakeOrderedAndProject [total_sales,i_manufact_id] WholeStageCodegen (17) HashAggregate [i_manufact_id,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] - Project [ws_bill_addr_sk,ws_ext_sales_price,i_manufact_id] - BroadcastHashJoin [ws_item_sk,i_item_sk] + BroadcastHashJoin [ws_item_sk,i_item_sk] + Project [ws_item_sk,ws_ext_sales_price] + BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] Project [ws_item_sk,ws_bill_addr_sk,ws_ext_sales_price] BroadcastHashJoin [ws_sold_date_sk,d_date_sk] Filter [ws_sold_date_sk,ws_bill_addr_sk,ws_item_sk] @@ -96,6 +96,6 @@ TakeOrderedAndProject [total_sales,i_manufact_id] InputAdapter ReusedExchange [d_date_sk] #3 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #4 + ReusedExchange [ca_address_sk] #4 InputAdapter - ReusedExchange [ca_address_sk] #6 + ReusedExchange [i_item_sk,i_manufact_id] #5 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt index 55bd25c501..5a9c4715d4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt @@ -10,8 +10,8 @@ TakeOrderedAndProject (81) : +- * HashAggregate (23) : +- * Project (22) : +- * BroadcastHashJoin Inner BuildRight (21) - : :- * Project (16) - : : +- * BroadcastHashJoin Inner BuildRight (15) + : :- * Project (15) + : : +- * BroadcastHashJoin Inner BuildRight (14) : : :- Union (9) : : : :- * Project (4) : : : : +- * Filter (3) @@ -21,22 +21,22 @@ TakeOrderedAndProject (81) : : : +- * Filter (7) : : : +- * ColumnarToRow (6) : : : +- Scan parquet default.store_returns (5) - : : +- BroadcastExchange (14) - : : +- * Project (13) - : : +- * Filter (12) - : : +- * ColumnarToRow (11) - : : +- Scan parquet default.date_dim (10) + : : +- BroadcastExchange (13) + : : +- * Filter (12) + : : +- * ColumnarToRow (11) + : : +- Scan parquet default.store (10) : +- BroadcastExchange (20) - : +- * Filter (19) - : +- * ColumnarToRow (18) - : +- Scan parquet default.store (17) + : +- * Project (19) + : +- * Filter (18) + : +- * ColumnarToRow (17) + : +- Scan parquet default.date_dim (16) :- * HashAggregate (46) : +- Exchange (45) : +- * HashAggregate (44) : +- * Project (43) : +- * BroadcastHashJoin Inner BuildRight (42) - : :- * Project (37) - : : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * Project (40) + : : +- * BroadcastHashJoin Inner BuildRight (39) : : :- Union (34) : : : :- * Project (29) : : : : +- * Filter (28) @@ -46,18 +46,18 @@ TakeOrderedAndProject (81) : : : +- * Filter (32) : : : +- * ColumnarToRow (31) : : : +- Scan parquet default.catalog_returns (30) - : : +- ReusedExchange (35) - : +- BroadcastExchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.catalog_page (38) + : : +- BroadcastExchange (38) + : : +- * Filter (37) + : : +- * ColumnarToRow (36) + : : +- Scan parquet default.catalog_page (35) + : +- ReusedExchange (41) +- * HashAggregate (75) +- Exchange (74) +- * HashAggregate (73) +- * Project (72) +- * BroadcastHashJoin Inner BuildRight (71) - :- * Project (66) - : +- * BroadcastHashJoin Inner BuildRight (65) + :- * Project (69) + : +- * BroadcastHashJoin Inner BuildRight (68) : :- Union (63) : : :- * Project (50) : : : +- * Filter (49) @@ -75,11 +75,11 @@ TakeOrderedAndProject (81) : : +- * Filter (58) : : +- * ColumnarToRow (57) : : +- Scan parquet default.web_sales (56) - : +- ReusedExchange (64) - +- BroadcastExchange (70) - +- * Filter (69) - +- * ColumnarToRow (68) - +- Scan parquet default.web_site (67) + : +- BroadcastExchange (67) + : +- * Filter (66) + : +- * ColumnarToRow (65) + : +- Scan parquet default.web_site (64) + +- ReusedExchange (70) (1) Scan parquet default.store_sales @@ -119,81 +119,81 @@ Input [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_los (9) Union -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] +(10) Scan parquet default.store +Output [2]: [s_store_sk#21, s_store_id#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] +Input [2]: [s_store_sk#21, s_store_id#22] (12) Filter [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 11192)) AND (d_date#22 <= 11206)) AND isnotnull(d_date_sk#21)) +Input [2]: [s_store_sk#21, s_store_id#22] +Condition : isnotnull(s_store_sk#21) -(13) Project [codegen id : 3] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] +(13) BroadcastExchange +Input [2]: [s_store_sk#21, s_store_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(14) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#6] -Right keys [1]: [cast(d_date_sk#21 as bigint)] +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [store_sk#5] +Right keys [1]: [cast(s_store_sk#21 as bigint)] Join condition: None -(16) Project [codegen id : 5] -Output [5]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10] -Input [7]: [store_sk#5, date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, d_date_sk#21] +(15) Project [codegen id : 5] +Output [6]: [date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Input [8]: [store_sk#5, date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_sk#21, s_store_id#22] -(17) Scan parquet default.store -Output [2]: [s_store_sk#24, s_store_id#25] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_date#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#24, d_date#25] -(18) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] +(18) Filter [codegen id : 4] +Input [2]: [d_date_sk#24, d_date#25] +Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 11192)) AND (d_date#25 <= 11206)) AND isnotnull(d_date_sk#24)) -(19) Filter [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) +(19) Project [codegen id : 4] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (20) BroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#5] -Right keys [1]: [cast(s_store_sk#24 as bigint)] +Left keys [1]: [date_sk#6] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (22) Project [codegen id : 5] -Output [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Input [7]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_sk#24, s_store_id#25] +Output [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Input [7]: [date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22, d_date_sk#24] (23) HashAggregate [codegen id : 5] -Input [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Keys [1]: [s_store_id#25] +Input [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Keys [1]: [s_store_id#22] Functions [4]: [partial_sum(UnscaledValue(sales_price#7)), partial_sum(UnscaledValue(return_amt#9)), partial_sum(UnscaledValue(profit#8)), partial_sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum#27, sum#28, sum#29, sum#30] -Results [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] +Results [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] (24) Exchange -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_id#25, 5), true, [id=#35] +Input [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] +Arguments: hashpartitioning(s_store_id#22, 5), ENSURE_REQUIREMENTS, [id=#35] (25) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Keys [1]: [s_store_id#25] +Input [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] +Keys [1]: [s_store_id#22] Functions [4]: [sum(UnscaledValue(sales_price#7)), sum(UnscaledValue(return_amt#9)), sum(UnscaledValue(profit#8)), sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#7))#36, sum(UnscaledValue(return_amt#9))#37, sum(UnscaledValue(profit#8))#38, sum(UnscaledValue(net_loss#10))#39] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS RETURNS#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#25) AS id#44] +Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#40, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS RETURNS#41, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#42, store channel AS channel#43, concat(store, s_store_id#22) AS id#44] (26) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] @@ -233,44 +233,44 @@ Input [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, (34) Union -(35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#50] -Right keys [1]: [d_date_sk#21] -Join condition: None - -(37) Project [codegen id : 11] -Output [5]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54] -Input [7]: [page_sk#49, date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, d_date_sk#21] - -(38) Scan parquet default.catalog_page +(35) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 9] Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(40) Filter [codegen id : 10] +(37) Filter [codegen id : 9] Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(41) BroadcastExchange +(38) BroadcastExchange Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#67] -(42) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#49] Right keys [1]: [cp_catalog_page_sk#65] Join condition: None +(40) Project [codegen id : 11] +Output [6]: [date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] +Input [8]: [page_sk#49, date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(41) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#24] + +(42) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#50] +Right keys [1]: [d_date_sk#24] +Join condition: None + (43) Project [codegen id : 11] Output [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] -Input [7]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_sk#65, cp_catalog_page_id#66] +Input [7]: [date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66, d_date_sk#24] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] @@ -281,7 +281,7 @@ Results [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] (45) Exchange Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] -Arguments: hashpartitioning(cp_catalog_page_id#66, 5), true, [id=#76] +Arguments: hashpartitioning(cp_catalog_page_id#66, 5), ENSURE_REQUIREMENTS, [id=#76] (46) HashAggregate [codegen id : 12] Input [5]: [cp_catalog_page_id#66, sum#72, sum#73, sum#74, sum#75] @@ -324,7 +324,7 @@ Condition : isnotnull(wr_returned_date_sk#96) (54) Exchange Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] -Arguments: hashpartitioning(wr_item_sk#97, wr_order_number#98, 5), true, [id=#101] +Arguments: hashpartitioning(wr_item_sk#97, wr_order_number#98, 5), ENSURE_REQUIREMENTS, [id=#101] (55) Sort [codegen id : 15] Input [5]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return_amt#99, wr_net_loss#100] @@ -346,7 +346,7 @@ Condition : ((isnotnull(ws_item_sk#102) AND isnotnull(ws_order_number#103)) AND (59) Exchange Input [3]: [ws_item_sk#102, ws_web_site_sk#87, ws_order_number#103] -Arguments: hashpartitioning(cast(ws_item_sk#102 as bigint), cast(ws_order_number#103 as bigint), 5), true, [id=#104] +Arguments: hashpartitioning(cast(ws_item_sk#102 as bigint), cast(ws_order_number#103 as bigint), 5), ENSURE_REQUIREMENTS, [id=#104] (60) Sort [codegen id : 17] Input [3]: [ws_item_sk#102, ws_web_site_sk#87, ws_order_number#103] @@ -363,44 +363,44 @@ Input [8]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (63) Union -(64) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(65) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#91] -Right keys [1]: [cast(d_date_sk#21 as bigint)] -Join condition: None - -(66) Project [codegen id : 21] -Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] -Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#21] - -(67) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 20] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#111, web_site_id#112] -(69) Filter [codegen id : 20] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(70) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#113] -(71) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#90] Right keys [1]: [web_site_sk#111] Join condition: None +(69) Project [codegen id : 21] +Output [6]: [date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] +Input [8]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#111, web_site_id#112] + +(70) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#24] + +(71) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [date_sk#91] +Right keys [1]: [cast(d_date_sk#24 as bigint)] +Join condition: None + (72) Project [codegen id : 21] Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] -Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#111, web_site_id#112] +Input [7]: [date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112, d_date_sk#24] (73) HashAggregate [codegen id : 21] Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] @@ -411,7 +411,7 @@ Results [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] (74) Exchange Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] -Arguments: hashpartitioning(web_site_id#112, 5), true, [id=#122] +Arguments: hashpartitioning(web_site_id#112, 5), ENSURE_REQUIREMENTS, [id=#122] (75) HashAggregate [codegen id : 22] Input [5]: [web_site_id#112, sum#118, sum#119, sum#120, sum#121] @@ -435,7 +435,7 @@ Results [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, (79) Exchange Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] -Arguments: hashpartitioning(channel#132, id#133, spark_grouping_id#134, 5), true, [id=#147] +Arguments: hashpartitioning(channel#132, id#133, spark_grouping_id#134, 5), ENSURE_REQUIREMENTS, [id=#147] (80) HashAggregate [codegen id : 24] Input [9]: [channel#132, id#133, spark_grouping_id#134, sum#141, isEmpty#142, sum#143, isEmpty#144, sum#145, isEmpty#146] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt index 80b07a3712..2db6cf7677 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/simplified.txt @@ -15,9 +15,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] InputAdapter Union WholeStageCodegen (1) @@ -35,18 +35,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #3 WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] WholeStageCodegen (12) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] InputAdapter @@ -54,9 +54,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] InputAdapter Union WholeStageCodegen (7) @@ -72,14 +72,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #6 + WholeStageCodegen (9) + Filter [cp_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (10) - Filter [cp_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [d_date_sk] #4 WholeStageCodegen (22) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,RETURNS,profit,channel,id,sum,sum,sum,sum] InputAdapter @@ -87,9 +87,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] InputAdapter Union WholeStageCodegen (13) @@ -122,11 +122,11 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter - ReusedExchange [d_date_sk] #3 + BroadcastExchange #10 + WholeStageCodegen (19) + Filter [web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_site [web_site_sk,web_site_id] InputAdapter - BroadcastExchange #10 - WholeStageCodegen (20) - Filter [web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_site_id] + ReusedExchange [d_date_sk] #4 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt index d7a8c10328..6492918d3a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/explain.txt @@ -6,117 +6,117 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 2000)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [2]: [d_date_sk#1, d_year#2] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [2]: [d_date_sk#1, d_year#2] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 1)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [3]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7] +Input [5]: [d_date_sk#1, d_year#2, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,1), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 2000)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 1)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [2]: [d_date_sk#9, d_year#10] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [2]: [d_date_sk#9, d_year#10] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9, d_year#10] +Output [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [6]: [d_year#2, ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [4]: [d_year#10, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [4]: [d_year#2, ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] +Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(d_year#10, i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(d_year#2, i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] -Keys [3]: [d_year#10, i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [4]: [d_year#10, i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Keys [3]: [d_year#2, i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [4]: [d_year#2, i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject -Input [4]: [d_year#10, brand_id#17, brand#18, ext_price#19] -Arguments: 100, [d_year#10 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#10, brand_id#17, brand#18, ext_price#19] +Input [4]: [d_year#2, brand_id#17, brand#18, ext_price#19] +Arguments: 100, [d_year#2 ASC NULLS FIRST, ext_price#19 DESC NULLS LAST, brand_id#17 ASC NULLS FIRST], [d_year#2, brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt index 8ed500d843..f4aaf3df75 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q52.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [d_year,ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [d_year,i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [d_year,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [d_year,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk,d_year] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk,d_year] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt index d78565986b..a504149b00 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt @@ -1,494 +1,474 @@ == Physical Plan == -TakeOrderedAndProject (71) -+- * HashAggregate (70) - +- Exchange (69) - +- * HashAggregate (68) - +- * HashAggregate (67) - +- Exchange (66) - +- * HashAggregate (65) - +- * Project (64) - +- * BroadcastHashJoin Inner BuildRight (63) - :- * Project (57) - : +- * BroadcastHashJoin Inner BuildRight (56) - : :- * Project (51) - : : +- * SortMergeJoin Inner (50) - : : :- * Sort (44) - : : : +- Exchange (43) - : : : +- * Project (42) - : : : +- * SortMergeJoin Inner (41) - : : : :- * Sort (35) - : : : : +- * HashAggregate (34) - : : : : +- * HashAggregate (33) - : : : : +- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (16) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : : : : :- Union (9) - : : : : : : : :- * Project (4) - : : : : : : : : +- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- * Project (8) - : : : : : : : +- * Filter (7) - : : : : : : : +- * ColumnarToRow (6) - : : : : : : : +- Scan parquet default.web_sales (5) - : : : : : : +- BroadcastExchange (14) - : : : : : : +- * Project (13) - : : : : : : +- * Filter (12) - : : : : : : +- * ColumnarToRow (11) - : : : : : : +- Scan parquet default.item (10) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Project (20) - : : : : : +- * Filter (19) - : : : : : +- * ColumnarToRow (18) - : : : : : +- Scan parquet default.date_dim (17) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.customer (26) - : : : +- * Sort (40) - : : : +- Exchange (39) - : : : +- * Filter (38) - : : : +- * ColumnarToRow (37) - : : : +- Scan parquet default.store_sales (36) - : : +- * Sort (49) - : : +- Exchange (48) - : : +- * Filter (47) - : : +- * ColumnarToRow (46) - : : +- Scan parquet default.customer_address (45) - : +- BroadcastExchange (55) - : +- * Filter (54) - : +- * ColumnarToRow (53) - : +- Scan parquet default.store (52) - +- BroadcastExchange (62) - +- * Project (61) - +- * Filter (60) - +- * ColumnarToRow (59) - +- Scan parquet default.date_dim (58) - - -(1) Scan parquet default.catalog_sales -Output [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] +TakeOrderedAndProject (67) ++- * HashAggregate (66) + +- Exchange (65) + +- * HashAggregate (64) + +- * HashAggregate (63) + +- * HashAggregate (62) + +- * Project (61) + +- * SortMergeJoin Inner (60) + :- * Sort (47) + : +- * Project (46) + : +- * BroadcastHashJoin Inner BuildLeft (45) + : :- BroadcastExchange (10) + : : +- * Project (9) + : : +- * BroadcastHashJoin Inner BuildRight (8) + : : :- * Filter (3) + : : : +- * ColumnarToRow (2) + : : : +- Scan parquet default.customer_address (1) + : : +- BroadcastExchange (7) + : : +- * Filter (6) + : : +- * ColumnarToRow (5) + : : +- Scan parquet default.store (4) + : +- * HashAggregate (44) + : +- * HashAggregate (43) + : +- * Project (42) + : +- * SortMergeJoin Inner (41) + : :- * Sort (35) + : : +- Exchange (34) + : : +- * Project (33) + : : +- * BroadcastHashJoin Inner BuildRight (32) + : : :- * Project (26) + : : : +- * BroadcastHashJoin Inner BuildRight (25) + : : : :- Union (19) + : : : : :- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.catalog_sales (11) + : : : : +- * Project (18) + : : : : +- * Filter (17) + : : : : +- * ColumnarToRow (16) + : : : : +- Scan parquet default.web_sales (15) + : : : +- BroadcastExchange (24) + : : : +- * Project (23) + : : : +- * Filter (22) + : : : +- * ColumnarToRow (21) + : : : +- Scan parquet default.date_dim (20) + : : +- BroadcastExchange (31) + : : +- * Project (30) + : : +- * Filter (29) + : : +- * ColumnarToRow (28) + : : +- Scan parquet default.item (27) + : +- * Sort (40) + : +- Exchange (39) + : +- * Filter (38) + : +- * ColumnarToRow (37) + : +- Scan parquet default.customer (36) + +- * Sort (59) + +- Exchange (58) + +- * Project (57) + +- * BroadcastHashJoin Inner BuildRight (56) + :- * Filter (50) + : +- * ColumnarToRow (49) + : +- Scan parquet default.store_sales (48) + +- BroadcastExchange (55) + +- * Project (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.date_dim (51) + + +(1) Scan parquet default.customer_address +Output [3]: [ca_address_sk#1, ca_county#2, ca_state#3] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] +ReadSchema: struct + +(2) ColumnarToRow [codegen id : 2] +Input [3]: [ca_address_sk#1, ca_county#2, ca_state#3] + +(3) Filter [codegen id : 2] +Input [3]: [ca_address_sk#1, ca_county#2, ca_state#3] +Condition : ((isnotnull(ca_address_sk#1) AND isnotnull(ca_county#2)) AND isnotnull(ca_state#3)) + +(4) Scan parquet default.store +Output [2]: [s_county#4, s_state#5] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] +ReadSchema: struct + +(5) ColumnarToRow [codegen id : 1] +Input [2]: [s_county#4, s_state#5] + +(6) Filter [codegen id : 1] +Input [2]: [s_county#4, s_state#5] +Condition : (isnotnull(s_county#4) AND isnotnull(s_state#5)) + +(7) BroadcastExchange +Input [2]: [s_county#4, s_state#5] +Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [id=#6] + +(8) BroadcastHashJoin [codegen id : 2] +Left keys [2]: [ca_county#2, ca_state#3] +Right keys [2]: [s_county#4, s_state#5] +Join condition: None + +(9) Project [codegen id : 2] +Output [1]: [ca_address_sk#1] +Input [5]: [ca_address_sk#1, ca_county#2, ca_state#3, s_county#4, s_state#5] + +(10) BroadcastExchange +Input [1]: [ca_address_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] + +(11) Scan parquet default.catalog_sales +Output [3]: [cs_sold_date_sk#8, cs_bill_customer_sk#9, cs_item_sk#10] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_customer_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] +(12) ColumnarToRow [codegen id : 3] +Input [3]: [cs_sold_date_sk#8, cs_bill_customer_sk#9, cs_item_sk#10] -(3) Filter [codegen id : 1] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] -Condition : ((isnotnull(cs_item_sk#3) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_bill_customer_sk#2)) +(13) Filter [codegen id : 3] +Input [3]: [cs_sold_date_sk#8, cs_bill_customer_sk#9, cs_item_sk#10] +Condition : ((isnotnull(cs_item_sk#10) AND isnotnull(cs_sold_date_sk#8)) AND isnotnull(cs_bill_customer_sk#9)) -(4) Project [codegen id : 1] -Output [3]: [cs_sold_date_sk#1 AS sold_date_sk#4, cs_bill_customer_sk#2 AS customer_sk#5, cs_item_sk#3 AS item_sk#6] -Input [3]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#3] +(14) Project [codegen id : 3] +Output [3]: [cs_sold_date_sk#8 AS sold_date_sk#11, cs_bill_customer_sk#9 AS customer_sk#12, cs_item_sk#10 AS item_sk#13] +Input [3]: [cs_sold_date_sk#8, cs_bill_customer_sk#9, cs_item_sk#10] -(5) Scan parquet default.web_sales -Output [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] +(15) Scan parquet default.web_sales +Output [3]: [ws_sold_date_sk#14, ws_item_sk#15, ws_bill_customer_sk#16] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_customer_sk)] ReadSchema: struct -(6) ColumnarToRow [codegen id : 2] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] +(16) ColumnarToRow [codegen id : 4] +Input [3]: [ws_sold_date_sk#14, ws_item_sk#15, ws_bill_customer_sk#16] -(7) Filter [codegen id : 2] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] -Condition : ((isnotnull(ws_item_sk#8) AND isnotnull(ws_sold_date_sk#7)) AND isnotnull(ws_bill_customer_sk#9)) +(17) Filter [codegen id : 4] +Input [3]: [ws_sold_date_sk#14, ws_item_sk#15, ws_bill_customer_sk#16] +Condition : ((isnotnull(ws_item_sk#15) AND isnotnull(ws_sold_date_sk#14)) AND isnotnull(ws_bill_customer_sk#16)) -(8) Project [codegen id : 2] -Output [3]: [ws_sold_date_sk#7 AS sold_date_sk#10, ws_bill_customer_sk#9 AS customer_sk#11, ws_item_sk#8 AS item_sk#12] -Input [3]: [ws_sold_date_sk#7, ws_item_sk#8, ws_bill_customer_sk#9] +(18) Project [codegen id : 4] +Output [3]: [ws_sold_date_sk#14 AS sold_date_sk#17, ws_bill_customer_sk#16 AS customer_sk#18, ws_item_sk#15 AS item_sk#19] +Input [3]: [ws_sold_date_sk#14, ws_item_sk#15, ws_bill_customer_sk#16] -(9) Union +(19) Union -(10) Scan parquet default.item -Output [3]: [i_item_sk#13, i_class#14, i_category#15] +(20) Scan parquet default.date_dim +Output [3]: [d_date_sk#20, d_year#21, d_moy#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] +ReadSchema: struct -(11) ColumnarToRow [codegen id : 3] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] +(21) ColumnarToRow [codegen id : 5] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -(12) Filter [codegen id : 3] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] -Condition : ((((isnotnull(i_category#15) AND isnotnull(i_class#14)) AND (i_category#15 = Women)) AND (i_class#14 = maternity)) AND isnotnull(i_item_sk#13)) +(22) Filter [codegen id : 5] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] +Condition : ((((isnotnull(d_moy#22) AND isnotnull(d_year#21)) AND (d_moy#22 = 12)) AND (d_year#21 = 1998)) AND isnotnull(d_date_sk#20)) -(13) Project [codegen id : 3] -Output [1]: [i_item_sk#13] -Input [3]: [i_item_sk#13, i_class#14, i_category#15] +(23) Project [codegen id : 5] +Output [1]: [d_date_sk#20] +Input [3]: [d_date_sk#20, d_year#21, d_moy#22] -(14) BroadcastExchange -Input [1]: [i_item_sk#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(24) BroadcastExchange +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [item_sk#6] -Right keys [1]: [i_item_sk#13] +(25) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [sold_date_sk#11] +Right keys [1]: [d_date_sk#20] Join condition: None -(16) Project [codegen id : 5] -Output [2]: [sold_date_sk#4, customer_sk#5] -Input [4]: [sold_date_sk#4, customer_sk#5, item_sk#6, i_item_sk#13] +(26) Project [codegen id : 7] +Output [2]: [customer_sk#12, item_sk#13] +Input [4]: [sold_date_sk#11, customer_sk#12, item_sk#13, d_date_sk#20] -(17) Scan parquet default.date_dim -Output [3]: [d_date_sk#17, d_year#18, d_moy#19] +(27) Scan parquet default.item +Output [3]: [i_item_sk#24, i_class#25, i_category#26] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,12), EqualTo(d_year,1998), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_category), IsNotNull(i_class), EqualTo(i_category,Women), EqualTo(i_class,maternity), IsNotNull(i_item_sk)] +ReadSchema: struct -(18) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +(28) ColumnarToRow [codegen id : 6] +Input [3]: [i_item_sk#24, i_class#25, i_category#26] -(19) Filter [codegen id : 4] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] -Condition : ((((isnotnull(d_moy#19) AND isnotnull(d_year#18)) AND (d_moy#19 = 12)) AND (d_year#18 = 1998)) AND isnotnull(d_date_sk#17)) +(29) Filter [codegen id : 6] +Input [3]: [i_item_sk#24, i_class#25, i_category#26] +Condition : ((((isnotnull(i_category#26) AND isnotnull(i_class#25)) AND (i_category#26 = Women)) AND (i_class#25 = maternity)) AND isnotnull(i_item_sk#24)) -(20) Project [codegen id : 4] -Output [1]: [d_date_sk#17] -Input [3]: [d_date_sk#17, d_year#18, d_moy#19] +(30) Project [codegen id : 6] +Output [1]: [i_item_sk#24] +Input [3]: [i_item_sk#24, i_class#25, i_category#26] -(21) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] +(31) BroadcastExchange +Input [1]: [i_item_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(22) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [sold_date_sk#4] -Right keys [1]: [d_date_sk#17] +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#13] +Right keys [1]: [i_item_sk#24] Join condition: None -(23) Project [codegen id : 5] -Output [1]: [customer_sk#5] -Input [3]: [sold_date_sk#4, customer_sk#5, d_date_sk#17] +(33) Project [codegen id : 7] +Output [1]: [customer_sk#12] +Input [3]: [customer_sk#12, item_sk#13, i_item_sk#24] -(24) Exchange -Input [1]: [customer_sk#5] -Arguments: hashpartitioning(customer_sk#5, 5), true, [id=#21] +(34) Exchange +Input [1]: [customer_sk#12] +Arguments: hashpartitioning(customer_sk#12, 5), ENSURE_REQUIREMENTS, [id=#28] -(25) Sort [codegen id : 6] -Input [1]: [customer_sk#5] -Arguments: [customer_sk#5 ASC NULLS FIRST], false, 0 +(35) Sort [codegen id : 8] +Input [1]: [customer_sk#12] +Arguments: [customer_sk#12 ASC NULLS FIRST], false, 0 -(26) Scan parquet default.customer -Output [2]: [c_customer_sk#22, c_current_addr_sk#23] +(36) Scan parquet default.customer +Output [2]: [c_customer_sk#29, c_current_addr_sk#30] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] +(37) ColumnarToRow [codegen id : 9] +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] -(28) Filter [codegen id : 7] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Condition : (isnotnull(c_customer_sk#22) AND isnotnull(c_current_addr_sk#23)) +(38) Filter [codegen id : 9] +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] +Condition : (isnotnull(c_customer_sk#29) AND isnotnull(c_current_addr_sk#30)) -(29) Exchange -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#24] +(39) Exchange +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] +Arguments: hashpartitioning(c_customer_sk#29, 5), ENSURE_REQUIREMENTS, [id=#31] -(30) Sort [codegen id : 8] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 10] +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 9] -Left keys [1]: [customer_sk#5] -Right keys [1]: [c_customer_sk#22] +(41) SortMergeJoin +Left keys [1]: [customer_sk#12] +Right keys [1]: [c_customer_sk#29] Join condition: None -(32) Project [codegen id : 9] -Output [2]: [c_customer_sk#22, c_current_addr_sk#23] -Input [3]: [customer_sk#5, c_customer_sk#22, c_current_addr_sk#23] +(42) Project +Output [2]: [c_customer_sk#29, c_current_addr_sk#30] +Input [3]: [customer_sk#12, c_customer_sk#29, c_current_addr_sk#30] -(33) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Keys [2]: [c_customer_sk#22, c_current_addr_sk#23] +(43) HashAggregate +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] +Keys [2]: [c_customer_sk#29, c_current_addr_sk#30] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#22, c_current_addr_sk#23] +Results [2]: [c_customer_sk#29, c_current_addr_sk#30] -(34) HashAggregate [codegen id : 9] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Keys [2]: [c_customer_sk#22, c_current_addr_sk#23] +(44) HashAggregate +Input [2]: [c_customer_sk#29, c_current_addr_sk#30] +Keys [2]: [c_customer_sk#29, c_current_addr_sk#30] Functions: [] Aggregate Attributes: [] -Results [2]: [c_customer_sk#22, c_current_addr_sk#23] +Results [2]: [c_customer_sk#29, c_current_addr_sk#30] -(35) Sort [codegen id : 9] -Input [2]: [c_customer_sk#22, c_current_addr_sk#23] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +(45) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [ca_address_sk#1] +Right keys [1]: [c_current_addr_sk#30] +Join condition: None -(36) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] +(46) Project [codegen id : 11] +Output [1]: [c_customer_sk#29] +Input [3]: [ca_address_sk#1, c_customer_sk#29, c_current_addr_sk#30] + +(47) Sort [codegen id : 11] +Input [1]: [c_customer_sk#29] +Arguments: [c_customer_sk#29 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#32, ss_customer_sk#33, ss_ext_sales_price#34] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(37) ColumnarToRow [codegen id : 10] -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] - -(38) Filter [codegen id : 10] -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Condition : (isnotnull(ss_customer_sk#26) AND isnotnull(ss_sold_date_sk#25)) - -(39) Exchange -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Arguments: hashpartitioning(ss_customer_sk#26, 5), true, [id=#28] - -(40) Sort [codegen id : 11] -Input [3]: [ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] -Arguments: [ss_customer_sk#26 ASC NULLS FIRST], false, 0 - -(41) SortMergeJoin [codegen id : 12] -Left keys [1]: [c_customer_sk#22] -Right keys [1]: [ss_customer_sk#26] -Join condition: None - -(42) Project [codegen id : 12] -Output [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] -Input [5]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_customer_sk#26, ss_ext_sales_price#27] +(49) ColumnarToRow [codegen id : 13] +Input [3]: [ss_sold_date_sk#32, ss_customer_sk#33, ss_ext_sales_price#34] -(43) Exchange -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] -Arguments: hashpartitioning(c_current_addr_sk#23, 5), true, [id=#29] +(50) Filter [codegen id : 13] +Input [3]: [ss_sold_date_sk#32, ss_customer_sk#33, ss_ext_sales_price#34] +Condition : (isnotnull(ss_customer_sk#33) AND isnotnull(ss_sold_date_sk#32)) -(44) Sort [codegen id : 13] -Input [4]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27] -Arguments: [c_current_addr_sk#23 ASC NULLS FIRST], false, 0 - -(45) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_county#31, ca_state#32] +(51) Scan parquet default.date_dim +Output [2]: [d_date_sk#20, d_month_seq#35] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_county), IsNotNull(ca_state)] -ReadSchema: struct - -(46) ColumnarToRow [codegen id : 14] -Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] - -(47) Filter [codegen id : 14] -Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] -Condition : ((isnotnull(ca_address_sk#30) AND isnotnull(ca_county#31)) AND isnotnull(ca_state#32)) - -(48) Exchange -Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] -Arguments: hashpartitioning(ca_address_sk#30, 5), true, [id=#33] - -(49) Sort [codegen id : 15] -Input [3]: [ca_address_sk#30, ca_county#31, ca_state#32] -Arguments: [ca_address_sk#30 ASC NULLS FIRST], false, 0 - -(50) SortMergeJoin [codegen id : 18] -Left keys [1]: [c_current_addr_sk#23] -Right keys [1]: [ca_address_sk#30] -Join condition: None +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] +ReadSchema: struct -(51) Project [codegen id : 18] -Output [5]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#31, ca_state#32] -Input [7]: [c_customer_sk#22, c_current_addr_sk#23, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_address_sk#30, ca_county#31, ca_state#32] +(52) ColumnarToRow [codegen id : 12] +Input [2]: [d_date_sk#20, d_month_seq#35] -(52) Scan parquet default.store -Output [2]: [s_county#34, s_state#35] -Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_county), IsNotNull(s_state)] -ReadSchema: struct +(53) Filter [codegen id : 12] +Input [2]: [d_date_sk#20, d_month_seq#35] +Condition : (((isnotnull(d_month_seq#35) AND (d_month_seq#35 >= Subquery scalar-subquery#36, [id=#37])) AND (d_month_seq#35 <= Subquery scalar-subquery#38, [id=#39])) AND isnotnull(d_date_sk#20)) -(53) ColumnarToRow [codegen id : 16] -Input [2]: [s_county#34, s_state#35] - -(54) Filter [codegen id : 16] -Input [2]: [s_county#34, s_state#35] -Condition : (isnotnull(s_county#34) AND isnotnull(s_state#35)) +(54) Project [codegen id : 12] +Output [1]: [d_date_sk#20] +Input [2]: [d_date_sk#20, d_month_seq#35] (55) BroadcastExchange -Input [2]: [s_county#34, s_state#35] -Arguments: HashedRelationBroadcastMode(List(input[0, string, false], input[1, string, false]),false), [id=#36] +Input [1]: [d_date_sk#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#40] -(56) BroadcastHashJoin [codegen id : 18] -Left keys [2]: [ca_county#31, ca_state#32] -Right keys [2]: [s_county#34, s_state#35] +(56) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [ss_sold_date_sk#32] +Right keys [1]: [d_date_sk#20] Join condition: None -(57) Project [codegen id : 18] -Output [3]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27] -Input [7]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, ca_county#31, ca_state#32, s_county#34, s_state#35] - -(58) Scan parquet default.date_dim -Output [2]: [d_date_sk#17, d_month_seq#37] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_month_seq), IsNotNull(d_date_sk)] -ReadSchema: struct - -(59) ColumnarToRow [codegen id : 17] -Input [2]: [d_date_sk#17, d_month_seq#37] - -(60) Filter [codegen id : 17] -Input [2]: [d_date_sk#17, d_month_seq#37] -Condition : (((isnotnull(d_month_seq#37) AND (d_month_seq#37 >= Subquery scalar-subquery#38, [id=#39])) AND (d_month_seq#37 <= Subquery scalar-subquery#40, [id=#41])) AND isnotnull(d_date_sk#17)) +(57) Project [codegen id : 13] +Output [2]: [ss_customer_sk#33, ss_ext_sales_price#34] +Input [4]: [ss_sold_date_sk#32, ss_customer_sk#33, ss_ext_sales_price#34, d_date_sk#20] -(61) Project [codegen id : 17] -Output [1]: [d_date_sk#17] -Input [2]: [d_date_sk#17, d_month_seq#37] +(58) Exchange +Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] +Arguments: hashpartitioning(ss_customer_sk#33, 5), ENSURE_REQUIREMENTS, [id=#41] -(62) BroadcastExchange -Input [1]: [d_date_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#42] +(59) Sort [codegen id : 14] +Input [2]: [ss_customer_sk#33, ss_ext_sales_price#34] +Arguments: [ss_customer_sk#33 ASC NULLS FIRST], false, 0 -(63) BroadcastHashJoin [codegen id : 18] -Left keys [1]: [ss_sold_date_sk#25] -Right keys [1]: [d_date_sk#17] +(60) SortMergeJoin [codegen id : 15] +Left keys [1]: [c_customer_sk#29] +Right keys [1]: [ss_customer_sk#33] Join condition: None -(64) Project [codegen id : 18] -Output [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Input [4]: [c_customer_sk#22, ss_sold_date_sk#25, ss_ext_sales_price#27, d_date_sk#17] - -(65) HashAggregate [codegen id : 18] -Input [2]: [c_customer_sk#22, ss_ext_sales_price#27] -Keys [1]: [c_customer_sk#22] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum#43] -Results [2]: [c_customer_sk#22, sum#44] - -(66) Exchange -Input [2]: [c_customer_sk#22, sum#44] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#45] - -(67) HashAggregate [codegen id : 19] -Input [2]: [c_customer_sk#22, sum#44] -Keys [1]: [c_customer_sk#22] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#27))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#27))#46] -Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#27))#46,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#47] - -(68) HashAggregate [codegen id : 19] -Input [1]: [segment#47] -Keys [1]: [segment#47] +(61) Project [codegen id : 15] +Output [2]: [c_customer_sk#29, ss_ext_sales_price#34] +Input [3]: [c_customer_sk#29, ss_customer_sk#33, ss_ext_sales_price#34] + +(62) HashAggregate [codegen id : 15] +Input [2]: [c_customer_sk#29, ss_ext_sales_price#34] +Keys [1]: [c_customer_sk#29] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#34))] +Aggregate Attributes [1]: [sum#42] +Results [2]: [c_customer_sk#29, sum#43] + +(63) HashAggregate [codegen id : 15] +Input [2]: [c_customer_sk#29, sum#43] +Keys [1]: [c_customer_sk#29] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#34))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#34))#44] +Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#34))#44,17,2)) / 50.00), DecimalType(21,6), true) as int) AS segment#45] + +(64) HashAggregate [codegen id : 15] +Input [1]: [segment#45] +Keys [1]: [segment#45] Functions [1]: [partial_count(1)] -Aggregate Attributes [1]: [count#48] -Results [2]: [segment#47, count#49] +Aggregate Attributes [1]: [count#46] +Results [2]: [segment#45, count#47] -(69) Exchange -Input [2]: [segment#47, count#49] -Arguments: hashpartitioning(segment#47, 5), true, [id=#50] +(65) Exchange +Input [2]: [segment#45, count#47] +Arguments: hashpartitioning(segment#45, 5), ENSURE_REQUIREMENTS, [id=#48] -(70) HashAggregate [codegen id : 20] -Input [2]: [segment#47, count#49] -Keys [1]: [segment#47] +(66) HashAggregate [codegen id : 16] +Input [2]: [segment#45, count#47] +Keys [1]: [segment#45] Functions [1]: [count(1)] -Aggregate Attributes [1]: [count(1)#51] -Results [3]: [segment#47, count(1)#51 AS num_customers#52, (segment#47 * 50) AS segment_base#53] +Aggregate Attributes [1]: [count(1)#49] +Results [3]: [segment#45, count(1)#49 AS num_customers#50, (segment#45 * 50) AS segment_base#51] -(71) TakeOrderedAndProject -Input [3]: [segment#47, num_customers#52, segment_base#53] -Arguments: 100, [segment#47 ASC NULLS FIRST, num_customers#52 ASC NULLS FIRST], [segment#47, num_customers#52, segment_base#53] +(67) TakeOrderedAndProject +Input [3]: [segment#45, num_customers#50, segment_base#51] +Arguments: 100, [segment#45 ASC NULLS FIRST, num_customers#50 ASC NULLS FIRST], [segment#45, num_customers#50, segment_base#51] ===== Subqueries ===== -Subquery:1 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#38, [id=#39] -* HashAggregate (78) -+- Exchange (77) - +- * HashAggregate (76) - +- * Project (75) - +- * Filter (74) - +- * ColumnarToRow (73) - +- Scan parquet default.date_dim (72) +Subquery:1 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#36, [id=#37] +* HashAggregate (74) ++- Exchange (73) + +- * HashAggregate (72) + +- * Project (71) + +- * Filter (70) + +- * ColumnarToRow (69) + +- Scan parquet default.date_dim (68) -(72) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#18, d_moy#19] +(68) Scan parquet default.date_dim +Output [3]: [d_month_seq#35, d_year#21, d_moy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(73) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] +(69) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] -(74) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) +(70) Filter [codegen id : 1] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] +Condition : (((isnotnull(d_year#21) AND isnotnull(d_moy#22)) AND (d_year#21 = 1998)) AND (d_moy#22 = 12)) -(75) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 1) AS (d_month_seq + 1)#54] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] +(71) Project [codegen id : 1] +Output [1]: [(d_month_seq#35 + 1) AS (d_month_seq + 1)#52] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] -(76) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +(72) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 1)#52] +Keys [1]: [(d_month_seq + 1)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#52] -(77) Exchange -Input [1]: [(d_month_seq + 1)#54] -Arguments: hashpartitioning((d_month_seq + 1)#54, 5), true, [id=#55] +(73) Exchange +Input [1]: [(d_month_seq + 1)#52] +Arguments: hashpartitioning((d_month_seq + 1)#52, 5), ENSURE_REQUIREMENTS, [id=#53] -(78) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 1)#54] -Keys [1]: [(d_month_seq + 1)#54] +(74) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 1)#52] +Keys [1]: [(d_month_seq + 1)#52] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 1)#54] +Results [1]: [(d_month_seq + 1)#52] -Subquery:2 Hosting operator id = 60 Hosting Expression = Subquery scalar-subquery#40, [id=#41] -* HashAggregate (85) -+- Exchange (84) - +- * HashAggregate (83) - +- * Project (82) - +- * Filter (81) - +- * ColumnarToRow (80) - +- Scan parquet default.date_dim (79) +Subquery:2 Hosting operator id = 53 Hosting Expression = Subquery scalar-subquery#38, [id=#39] +* HashAggregate (81) ++- Exchange (80) + +- * HashAggregate (79) + +- * Project (78) + +- * Filter (77) + +- * ColumnarToRow (76) + +- Scan parquet default.date_dim (75) -(79) Scan parquet default.date_dim -Output [3]: [d_month_seq#37, d_year#18, d_moy#19] +(75) Scan parquet default.date_dim +Output [3]: [d_month_seq#35, d_year#21, d_moy#22] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,12)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] +(76) ColumnarToRow [codegen id : 1] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] -(81) Filter [codegen id : 1] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] -Condition : (((isnotnull(d_year#18) AND isnotnull(d_moy#19)) AND (d_year#18 = 1998)) AND (d_moy#19 = 12)) +(77) Filter [codegen id : 1] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] +Condition : (((isnotnull(d_year#21) AND isnotnull(d_moy#22)) AND (d_year#21 = 1998)) AND (d_moy#22 = 12)) -(82) Project [codegen id : 1] -Output [1]: [(d_month_seq#37 + 3) AS (d_month_seq + 3)#56] -Input [3]: [d_month_seq#37, d_year#18, d_moy#19] +(78) Project [codegen id : 1] +Output [1]: [(d_month_seq#35 + 3) AS (d_month_seq + 3)#54] +Input [3]: [d_month_seq#35, d_year#21, d_moy#22] -(83) HashAggregate [codegen id : 1] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +(79) HashAggregate [codegen id : 1] +Input [1]: [(d_month_seq + 3)#54] +Keys [1]: [(d_month_seq + 3)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#54] -(84) Exchange -Input [1]: [(d_month_seq + 3)#56] -Arguments: hashpartitioning((d_month_seq + 3)#56, 5), true, [id=#57] +(80) Exchange +Input [1]: [(d_month_seq + 3)#54] +Arguments: hashpartitioning((d_month_seq + 3)#54, 5), ENSURE_REQUIREMENTS, [id=#55] -(85) HashAggregate [codegen id : 2] -Input [1]: [(d_month_seq + 3)#56] -Keys [1]: [(d_month_seq + 3)#56] +(81) HashAggregate [codegen id : 2] +Input [1]: [(d_month_seq + 3)#54] +Keys [1]: [(d_month_seq + 3)#54] Functions: [] Aggregate Attributes: [] -Results [1]: [(d_month_seq + 3)#56] +Results [1]: [(d_month_seq + 3)#54] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt index cb7130f53c..3b0622cbf9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/simplified.txt @@ -1,142 +1,130 @@ TakeOrderedAndProject [segment,num_customers,segment_base] - WholeStageCodegen (20) + WholeStageCodegen (16) HashAggregate [segment,count] [count(1),num_customers,segment_base,count] InputAdapter Exchange [segment] #1 - WholeStageCodegen (19) + WholeStageCodegen (15) HashAggregate [segment] [count,count] HashAggregate [c_customer_sk,sum] [sum(UnscaledValue(ss_ext_sales_price)),segment,sum] - InputAdapter - Exchange [c_customer_sk] #2 - WholeStageCodegen (18) - HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] - Project [c_customer_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price] - BroadcastHashJoin [ca_county,ca_state,s_county,s_state] - Project [c_customer_sk,ss_sold_date_sk,ss_ext_sales_price,ca_county,ca_state] - SortMergeJoin [c_current_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (13) - Sort [c_current_addr_sk] + HashAggregate [c_customer_sk,ss_ext_sales_price] [sum,sum] + Project [c_customer_sk,ss_ext_sales_price] + SortMergeJoin [c_customer_sk,ss_customer_sk] + InputAdapter + WholeStageCodegen (11) + Sort [c_customer_sk] + Project [c_customer_sk] + BroadcastHashJoin [ca_address_sk,c_current_addr_sk] + InputAdapter + BroadcastExchange #2 + WholeStageCodegen (2) + Project [ca_address_sk] + BroadcastHashJoin [ca_county,ca_state,s_county,s_state] + Filter [ca_address_sk,ca_county,ca_state] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] InputAdapter - Exchange [c_current_addr_sk] #3 - WholeStageCodegen (12) - Project [c_customer_sk,c_current_addr_sk,ss_sold_date_sk,ss_ext_sales_price] - SortMergeJoin [c_customer_sk,ss_customer_sk] + BroadcastExchange #3 + WholeStageCodegen (1) + Filter [s_county,s_state] + ColumnarToRow InputAdapter - WholeStageCodegen (9) - Sort [c_customer_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - HashAggregate [c_customer_sk,c_current_addr_sk] - Project [c_customer_sk,c_current_addr_sk] - SortMergeJoin [customer_sk,c_customer_sk] - InputAdapter - WholeStageCodegen (6) - Sort [customer_sk] + Scan parquet default.store [s_county,s_state] + HashAggregate [c_customer_sk,c_current_addr_sk] + HashAggregate [c_customer_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk] + SortMergeJoin [customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (8) + Sort [customer_sk] + InputAdapter + Exchange [customer_sk] #4 + WholeStageCodegen (7) + Project [customer_sk] + BroadcastHashJoin [item_sk,i_item_sk] + Project [customer_sk,item_sk] + BroadcastHashJoin [sold_date_sk,d_date_sk] + InputAdapter + Union + WholeStageCodegen (3) + Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] + ColumnarToRow InputAdapter - Exchange [customer_sk] #4 - WholeStageCodegen (5) - Project [customer_sk] - BroadcastHashJoin [sold_date_sk,d_date_sk] - Project [sold_date_sk,customer_sk] - BroadcastHashJoin [item_sk,i_item_sk] - InputAdapter - Union - WholeStageCodegen (1) - Project [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - Filter [cs_item_sk,cs_sold_date_sk,cs_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] - WholeStageCodegen (2) - Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] - Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [i_item_sk] - Filter [i_category,i_class,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_class,i_category] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] - InputAdapter - WholeStageCodegen (8) - Sort [c_customer_sk] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_bill_customer_sk,cs_item_sk] + WholeStageCodegen (4) + Project [ws_sold_date_sk,ws_bill_customer_sk,ws_item_sk] + Filter [ws_item_sk,ws_sold_date_sk,ws_bill_customer_sk] + ColumnarToRow InputAdapter - Exchange [c_customer_sk] #7 - WholeStageCodegen (7) - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_current_addr_sk] - InputAdapter - WholeStageCodegen (11) - Sort [ss_customer_sk] - InputAdapter - Exchange [ss_customer_sk] #8 - WholeStageCodegen (10) - Filter [ss_customer_sk,ss_sold_date_sk] + Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_bill_customer_sk] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (6) + Project [i_item_sk] + Filter [i_category,i_class,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_sales_price] - InputAdapter - WholeStageCodegen (15) - Sort [ca_address_sk] - InputAdapter - Exchange [ca_address_sk] #9 - WholeStageCodegen (14) - Filter [ca_address_sk,ca_county,ca_state] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (16) - Filter [s_county,s_state] + Scan parquet default.item [i_item_sk,i_class,i_category] + InputAdapter + WholeStageCodegen (10) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #7 + WholeStageCodegen (9) + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer [c_customer_sk,c_current_addr_sk] + InputAdapter + WholeStageCodegen (14) + Sort [ss_customer_sk] + InputAdapter + Exchange [ss_customer_sk] #8 + WholeStageCodegen (13) + Project [ss_customer_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Filter [ss_customer_sk,ss_sold_date_sk] ColumnarToRow InputAdapter - Scan parquet default.store [s_county,s_state] - InputAdapter - BroadcastExchange #11 - WholeStageCodegen (17) - Project [d_date_sk] - Filter [d_month_seq,d_date_sk] - Subquery #1 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 1)] - InputAdapter - Exchange [(d_month_seq + 1)] #12 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 1)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - Subquery #2 - WholeStageCodegen (2) - HashAggregate [(d_month_seq + 3)] - InputAdapter - Exchange [(d_month_seq + 3)] #13 - WholeStageCodegen (1) - HashAggregate [(d_month_seq + 3)] - Project [d_month_seq] - Filter [d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_month_seq,d_year,d_moy] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_month_seq] + Scan parquet default.store_sales [ss_sold_date_sk,ss_customer_sk,ss_ext_sales_price] + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (12) + Project [d_date_sk] + Filter [d_month_seq,d_date_sk] + Subquery #1 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 1)] + InputAdapter + Exchange [(d_month_seq + 1)] #10 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 1)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + Subquery #2 + WholeStageCodegen (2) + HashAggregate [(d_month_seq + 3)] + InputAdapter + Exchange [(d_month_seq + 3)] #11 + WholeStageCodegen (1) + HashAggregate [(d_month_seq + 3)] + Project [d_month_seq] + Filter [d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_month_seq,d_year,d_moy] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_month_seq] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt index a1257cd292..b8d8aa358d 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/explain.txt @@ -6,115 +6,115 @@ TakeOrderedAndProject (21) +- * Project (17) +- * BroadcastHashJoin Inner BuildRight (16) :- * Project (10) - : +- * BroadcastHashJoin Inner BuildRight (9) - : :- * Filter (3) - : : +- * ColumnarToRow (2) - : : +- Scan parquet default.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.item (4) + : +- * BroadcastHashJoin Inner BuildLeft (9) + : :- BroadcastExchange (5) + : : +- * Project (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.date_dim (1) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.store_sales (6) +- BroadcastExchange (15) +- * Project (14) +- * Filter (13) +- * ColumnarToRow (12) - +- Scan parquet default.date_dim (11) + +- Scan parquet default.item (11) -(1) Scan parquet default.store_sales -Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(1) Scan parquet default.date_dim +Output [3]: [d_date_sk#1, d_year#2, d_moy#3] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +(2) ColumnarToRow [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(3) Filter [codegen id : 3] -Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -Condition : (isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_item_sk#2)) +(3) Filter [codegen id : 1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +Condition : ((((isnotnull(d_moy#3) AND isnotnull(d_year#2)) AND (d_moy#3 = 11)) AND (d_year#2 = 1999)) AND isnotnull(d_date_sk#1)) -(4) Scan parquet default.item -Output [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] -ReadSchema: struct +(4) Project [codegen id : 1] +Output [1]: [d_date_sk#1] +Input [3]: [d_date_sk#1, d_year#2, d_moy#3] -(5) ColumnarToRow [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(5) BroadcastExchange +Input [1]: [d_date_sk#1] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] -(6) Filter [codegen id : 1] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -Condition : ((isnotnull(i_manager_id#7) AND (i_manager_id#7 = 28)) AND isnotnull(i_item_sk#4)) +(6) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Batched: true +Location [not included in comparison]/{warehouse_dir}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] +(7) ColumnarToRow +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(8) BroadcastExchange -Input [3]: [i_item_sk#4, i_brand_id#5, i_brand#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +(8) Filter +Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +Condition : (isnotnull(ss_sold_date_sk#5) AND isnotnull(ss_item_sk#6)) (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#4] +Left keys [1]: [d_date_sk#1] +Right keys [1]: [ss_sold_date_sk#5] Join condition: None (10) Project [codegen id : 3] -Output [4]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [6]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#4, i_brand_id#5, i_brand#6] +Output [2]: [ss_item_sk#6, ss_ext_sales_price#7] +Input [4]: [d_date_sk#1, ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] -(11) Scan parquet default.date_dim -Output [3]: [d_date_sk#9, d_year#10, d_moy#11] +(11) Scan parquet default.item +Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_moy), IsNotNull(d_year), EqualTo(d_moy,11), EqualTo(d_year,1999), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [IsNotNull(i_manager_id), EqualTo(i_manager_id,28), IsNotNull(i_item_sk)] +ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (13) Filter [codegen id : 2] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] -Condition : ((((isnotnull(d_moy#11) AND isnotnull(d_year#10)) AND (d_moy#11 = 11)) AND (d_year#10 = 1999)) AND isnotnull(d_date_sk#9)) +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Condition : ((isnotnull(i_manager_id#11) AND (i_manager_id#11 = 28)) AND isnotnull(i_item_sk#8)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#9] -Input [3]: [d_date_sk#9, d_year#10, d_moy#11] +Output [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] (15) BroadcastExchange -Input [1]: [d_date_sk#9] +Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#9] +Left keys [1]: [ss_item_sk#6] +Right keys [1]: [i_item_sk#8] Join condition: None (17) Project [codegen id : 3] -Output [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Input [5]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_brand_id#5, i_brand#6, d_date_sk#9] +Output [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Input [5]: [ss_item_sk#6, ss_ext_sales_price#7, i_item_sk#8, i_brand_id#9, i_brand#10] (18) HashAggregate [codegen id : 3] -Input [3]: [ss_ext_sales_price#3, i_brand_id#5, i_brand#6] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Input [3]: [ss_ext_sales_price#7, i_brand_id#9, i_brand#10] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#7))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#6, i_brand_id#5, sum#14] +Results [3]: [i_brand#10, i_brand_id#9, sum#14] (19) Exchange -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), true, [id=#15] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Arguments: hashpartitioning(i_brand#10, i_brand_id#9, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -Input [3]: [i_brand#6, i_brand_id#5, sum#14] -Keys [2]: [i_brand#6, i_brand_id#5] -Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#16] -Results [3]: [i_brand_id#5 AS brand_id#17, i_brand#6 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#16,17,2) AS ext_price#19] +Input [3]: [i_brand#10, i_brand_id#9, sum#14] +Keys [2]: [i_brand#10, i_brand_id#9] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#7))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#7))#16] +Results [3]: [i_brand_id#9 AS brand_id#17, i_brand#10 AS brand#18, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#7))#16,17,2) AS ext_price#19] (21) TakeOrderedAndProject Input [3]: [brand_id#17, brand#18, ext_price#19] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt index b0d0e0d809..4f375c8067 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q55.sf100/simplified.txt @@ -6,26 +6,26 @@ TakeOrderedAndProject [ext_price,brand_id,brand] WholeStageCodegen (3) HashAggregate [i_brand,i_brand_id,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] - Project [ss_sold_date_sk,ss_ext_sales_price,i_brand_id,i_brand] - BroadcastHashJoin [ss_item_sk,i_item_sk] - Filter [ss_sold_date_sk,ss_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [ss_item_sk,i_item_sk] + Project [ss_item_sk,ss_ext_sales_price] + BroadcastHashJoin [d_date_sk,ss_sold_date_sk] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - Project [i_item_sk,i_brand_id,i_brand] - Filter [i_manager_id,i_item_sk] + Project [d_date_sk] + Filter [d_moy,d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Filter [ss_sold_date_sk,ss_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + Project [i_item_sk,i_brand_id,i_brand] + Filter [i_manager_id,i_item_sk] ColumnarToRow InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt index 3f8106c963..3007b11a1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1999), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 1999)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 1999)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt index 918508787c..b88505ad7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt index 6e757528a3..6813696266 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt @@ -1,343 +1,343 @@ == Physical Plan == TakeOrderedAndProject (61) +- * Project (60) - +- * BroadcastHashJoin Inner BuildRight (59) - :- * Project (38) - : +- * SortMergeJoin Inner (37) - : :- * Sort (11) - : : +- Exchange (10) - : : +- * Project (9) - : : +- * BroadcastHashJoin Inner BuildRight (8) - : : :- * Filter (3) - : : : +- * ColumnarToRow (2) - : : : +- Scan parquet default.customer (1) - : : +- BroadcastExchange (7) - : : +- * Filter (6) - : : +- * ColumnarToRow (5) - : : +- Scan parquet default.customer_address (4) - : +- * Sort (36) - : +- Exchange (35) - : +- * Filter (34) - : +- * HashAggregate (33) - : +- Exchange (32) - : +- * HashAggregate (31) - : +- * Project (30) - : +- * SortMergeJoin Inner (29) - : :- * Sort (23) - : : +- Exchange (22) - : : +- * Project (21) - : : +- * BroadcastHashJoin Inner BuildRight (20) - : : :- * Filter (14) - : : : +- * ColumnarToRow (13) - : : : +- Scan parquet default.catalog_returns (12) - : : +- BroadcastExchange (19) - : : +- * Project (18) - : : +- * Filter (17) - : : +- * ColumnarToRow (16) - : : +- Scan parquet default.date_dim (15) - : +- * Sort (28) - : +- Exchange (27) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- BroadcastExchange (58) - +- * Filter (57) - +- * HashAggregate (56) - +- Exchange (55) - +- * HashAggregate (54) - +- * HashAggregate (53) - +- Exchange (52) - +- * HashAggregate (51) - +- * Project (50) - +- * SortMergeJoin Inner (49) - :- * Sort (46) - : +- Exchange (45) - : +- * Project (44) - : +- * BroadcastHashJoin Inner BuildRight (43) - : :- * Filter (41) - : : +- * ColumnarToRow (40) - : : +- Scan parquet default.catalog_returns (39) - : +- ReusedExchange (42) - +- * Sort (48) - +- ReusedExchange (47) - - -(1) Scan parquet default.customer -Output [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] + +- * SortMergeJoin Inner (59) + :- * Sort (47) + : +- Exchange (46) + : +- * Project (45) + : +- * BroadcastHashJoin Inner BuildRight (44) + : :- * Filter (23) + : : +- * HashAggregate (22) + : : +- Exchange (21) + : : +- * HashAggregate (20) + : : +- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.catalog_returns (1) + : : : +- BroadcastExchange (8) + : : : +- * Project (7) + : : : +- * Filter (6) + : : : +- * ColumnarToRow (5) + : : : +- Scan parquet default.date_dim (4) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer_address (13) + : +- BroadcastExchange (43) + : +- * Filter (42) + : +- * HashAggregate (41) + : +- Exchange (40) + : +- * HashAggregate (39) + : +- * HashAggregate (38) + : +- Exchange (37) + : +- * HashAggregate (36) + : +- * Project (35) + : +- * SortMergeJoin Inner (34) + : :- * Sort (31) + : : +- Exchange (30) + : : +- * Project (29) + : : +- * BroadcastHashJoin Inner BuildRight (28) + : : :- * Filter (26) + : : : +- * ColumnarToRow (25) + : : : +- Scan parquet default.catalog_returns (24) + : : +- ReusedExchange (27) + : +- * Sort (33) + : +- ReusedExchange (32) + +- * Sort (58) + +- Exchange (57) + +- * Project (56) + +- * BroadcastHashJoin Inner BuildRight (55) + :- * Filter (50) + : +- * ColumnarToRow (49) + : +- Scan parquet default.customer (48) + +- BroadcastExchange (54) + +- * Filter (53) + +- * ColumnarToRow (52) + +- Scan parquet default.customer_address (51) + + +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] (3) Filter [codegen id : 2] -Input [6]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6] -Condition : (isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#3)) +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : ((isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_address -Output [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +(4) Scan parquet default.date_dim +Output [2]: [d_date_sk#5, d_year#6] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +Input [2]: [d_date_sk#5, d_year#6] (6) Filter [codegen id : 1] -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Condition : ((isnotnull(ca_state#14) AND (ca_state#14 = GA)) AND isnotnull(ca_address_sk#7)) +Input [2]: [d_date_sk#5, d_year#6] +Condition : ((isnotnull(d_year#6) AND (d_year#6 = 2000)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [2]: [d_date_sk#5, d_year#6] -(7) BroadcastExchange -Input [12]: [ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] -(8) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [c_current_addr_sk#3] -Right keys [1]: [ca_address_sk#7] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 2] -Output [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Input [18]: [c_customer_sk#1, c_customer_id#2, c_current_addr_sk#3, c_salutation#4, c_first_name#5, c_last_name#6, ca_address_sk#7, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(10) Exchange -Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: hashpartitioning(c_customer_sk#1, 5), true, [id=#20] +(11) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#8] -(11) Sort [codegen id : 3] -Input [16]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18] -Arguments: [c_customer_sk#1 ASC NULLS FIRST], false, 0 +(12) Sort [codegen id : 3] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(13) Scan parquet default.customer_address +Output [2]: [ca_address_sk#9, ca_state#10] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct - -(13) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] +ReadSchema: struct -(14) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : ((isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) AND isnotnull(cr_returning_customer_sk#22)) +(14) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] -(15) Scan parquet default.date_dim -Output [2]: [d_date_sk#25, d_year#26] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2000), IsNotNull(d_date_sk)] -ReadSchema: struct +(15) Filter [codegen id : 4] +Input [2]: [ca_address_sk#9, ca_state#10] +Condition : (isnotnull(ca_address_sk#9) AND isnotnull(ca_state#10)) -(16) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] +(16) Exchange +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] -(17) Filter [codegen id : 4] -Input [2]: [d_date_sk#25, d_year#26] -Condition : ((isnotnull(d_year#26) AND (d_year#26 = 2000)) AND isnotnull(d_date_sk#25)) +(17) Sort [codegen id : 5] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(18) Project [codegen id : 4] -Output [1]: [d_date_sk#25] -Input [2]: [d_date_sk#25, d_year#26] +(18) SortMergeJoin [codegen id : 6] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] +Join condition: None -(19) BroadcastExchange -Input [1]: [d_date_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] +(19) Project [codegen id : 6] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(20) HashAggregate [codegen id : 6] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#12] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] + +(21) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#14] + +(22) HashAggregate [codegen id : 15] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#13] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#15] +Results [3]: [cr_returning_customer_sk#2 AS ctr_customer_sk#16, ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#15,17,2) AS ctr_total_return#18] + +(23) Filter [codegen id : 15] +Input [3]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18] +Condition : isnotnull(ctr_total_return#18) + +(24) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Batched: true +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(20) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] -Join condition: None +(25) ColumnarToRow [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] -(21) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(26) Filter [codegen id : 8] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Condition : (isnotnull(cr_returned_date_sk#1) AND isnotnull(cr_returning_addr_sk#3)) -(22) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#28] +(27) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#5] -(23) Sort [codegen id : 6] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(28) BroadcastHashJoin [codegen id : 8] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] +Join condition: None -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#7, ca_state#14] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -ReadSchema: struct +(29) Project [codegen id : 8] +Output [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, d_date_sk#5] -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] +(30) Exchange +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: hashpartitioning(cr_returning_addr_sk#3, 5), ENSURE_REQUIREMENTS, [id=#19] -(26) Filter [codegen id : 7] -Input [2]: [ca_address_sk#7, ca_state#14] -Condition : (isnotnull(ca_address_sk#7) AND isnotnull(ca_state#14)) +(31) Sort [codegen id : 9] +Input [3]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +Arguments: [cr_returning_addr_sk#3 ASC NULLS FIRST], false, 0 -(27) Exchange -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: hashpartitioning(ca_address_sk#7, 5), true, [id=#29] +(32) ReusedExchange [Reuses operator id: 16] +Output [2]: [ca_address_sk#9, ca_state#10] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 +(33) Sort [codegen id : 11] +Input [2]: [ca_address_sk#9, ca_state#10] +Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(34) SortMergeJoin [codegen id : 12] +Left keys [1]: [cr_returning_addr_sk#3] +Right keys [1]: [ca_address_sk#9] Join condition: None -(30) Project [codegen id : 9] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(31) HashAggregate [codegen id : 9] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#30] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] - -(32) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#32] - -(33) HashAggregate [codegen id : 10] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#31] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#33] -Results [3]: [cr_returning_customer_sk#22 AS ctr_customer_sk#34, ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#33,17,2) AS ctr_total_return#36] - -(34) Filter [codegen id : 10] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Condition : isnotnull(ctr_total_return#36) - -(35) Exchange -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: hashpartitioning(ctr_customer_sk#34, 5), true, [id=#37] - -(36) Sort [codegen id : 11] -Input [3]: [ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] -Arguments: [ctr_customer_sk#34 ASC NULLS FIRST], false, 0 - -(37) SortMergeJoin [codegen id : 20] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [ctr_customer_sk#34] -Join condition: None +(35) Project [codegen id : 12] +Output [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Input [5]: [cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4, ca_address_sk#9, ca_state#10] + +(36) HashAggregate [codegen id : 12] +Input [3]: [cr_returning_customer_sk#2, cr_return_amt_inc_tax#4, ca_state#10] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum#20] +Results [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] + +(37) Exchange +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Arguments: hashpartitioning(cr_returning_customer_sk#2, ca_state#10, 5), ENSURE_REQUIREMENTS, [id=#22] + +(38) HashAggregate [codegen id : 13] +Input [3]: [cr_returning_customer_sk#2, ca_state#10, sum#21] +Keys [2]: [cr_returning_customer_sk#2, ca_state#10] +Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#4))#23] +Results [2]: [ca_state#10 AS ctr_state#17, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#4))#23,17,2) AS ctr_total_return#18] + +(39) HashAggregate [codegen id : 13] +Input [2]: [ctr_state#17, ctr_total_return#18] +Keys [1]: [ctr_state#17] +Functions [1]: [partial_avg(ctr_total_return#18)] +Aggregate Attributes [2]: [sum#24, count#25] +Results [3]: [ctr_state#17, sum#26, count#27] + +(40) Exchange +Input [3]: [ctr_state#17, sum#26, count#27] +Arguments: hashpartitioning(ctr_state#17, 5), ENSURE_REQUIREMENTS, [id=#28] + +(41) HashAggregate [codegen id : 14] +Input [3]: [ctr_state#17, sum#26, count#27] +Keys [1]: [ctr_state#17] +Functions [1]: [avg(ctr_total_return#18)] +Aggregate Attributes [1]: [avg(ctr_total_return#18)#29] +Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#18)#29) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17 AS ctr_state#17#31] + +(42) Filter [codegen id : 14] +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(43) BroadcastExchange +Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] +Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#32] + +(44) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [ctr_state#17] +Right keys [1]: [ctr_state#17#31] +Join condition: (cast(ctr_total_return#18 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30) + +(45) Project [codegen id : 15] +Output [2]: [ctr_customer_sk#16, ctr_total_return#18] +Input [5]: [ctr_customer_sk#16, ctr_state#17, ctr_total_return#18, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#30, ctr_state#17#31] + +(46) Exchange +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: hashpartitioning(ctr_customer_sk#16, 5), ENSURE_REQUIREMENTS, [id=#33] + +(47) Sort [codegen id : 16] +Input [2]: [ctr_customer_sk#16, ctr_total_return#18] +Arguments: [ctr_customer_sk#16 ASC NULLS FIRST], false, 0 + +(48) Scan parquet default.customer +Output [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct + +(49) ColumnarToRow [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] -(38) Project [codegen id : 20] -Output [17]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36] -Input [19]: [c_customer_sk#1, c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_customer_sk#34, ctr_state#35, ctr_total_return#36] +(50) Filter [codegen id : 18] +Input [6]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39] +Condition : (isnotnull(c_customer_sk#34) AND isnotnull(c_current_addr_sk#36)) -(39) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(51) Scan parquet default.customer_address +Output [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct -(40) ColumnarToRow [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] +(52) ColumnarToRow [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(41) Filter [codegen id : 13] -Input [4]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Condition : (isnotnull(cr_returned_date_sk#21) AND isnotnull(cr_returning_addr_sk#23)) +(53) Filter [codegen id : 17] +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Condition : ((isnotnull(ca_state#10) AND (ca_state#10 = GA)) AND isnotnull(ca_address_sk#9)) -(42) ReusedExchange [Reuses operator id: 19] -Output [1]: [d_date_sk#25] +(54) BroadcastExchange +Input [12]: [ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#50] -(43) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [cr_returned_date_sk#21] -Right keys [1]: [d_date_sk#25] +(55) BroadcastHashJoin [codegen id : 18] +Left keys [1]: [c_current_addr_sk#36] +Right keys [1]: [ca_address_sk#9] Join condition: None -(44) Project [codegen id : 13] -Output [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Input [5]: [cr_returned_date_sk#21, cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, d_date_sk#25] +(56) Project [codegen id : 18] +Output [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Input [18]: [c_customer_sk#34, c_customer_id#35, c_current_addr_sk#36, c_salutation#37, c_first_name#38, c_last_name#39, ca_address_sk#9, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] -(45) Exchange -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: hashpartitioning(cr_returning_addr_sk#23, 5), true, [id=#38] +(57) Exchange +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: hashpartitioning(c_customer_sk#34, 5), ENSURE_REQUIREMENTS, [id=#51] -(46) Sort [codegen id : 14] -Input [3]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24] -Arguments: [cr_returning_addr_sk#23 ASC NULLS FIRST], false, 0 +(58) Sort [codegen id : 19] +Input [16]: [c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] +Arguments: [c_customer_sk#34 ASC NULLS FIRST], false, 0 -(47) ReusedExchange [Reuses operator id: 27] -Output [2]: [ca_address_sk#7, ca_state#14] - -(48) Sort [codegen id : 16] -Input [2]: [ca_address_sk#7, ca_state#14] -Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 - -(49) SortMergeJoin [codegen id : 17] -Left keys [1]: [cr_returning_addr_sk#23] -Right keys [1]: [ca_address_sk#7] +(59) SortMergeJoin [codegen id : 20] +Left keys [1]: [ctr_customer_sk#16] +Right keys [1]: [c_customer_sk#34] Join condition: None -(50) Project [codegen id : 17] -Output [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Input [5]: [cr_returning_customer_sk#22, cr_returning_addr_sk#23, cr_return_amt_inc_tax#24, ca_address_sk#7, ca_state#14] - -(51) HashAggregate [codegen id : 17] -Input [3]: [cr_returning_customer_sk#22, cr_return_amt_inc_tax#24, ca_state#14] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [partial_sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum#39] -Results [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] - -(52) Exchange -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Arguments: hashpartitioning(cr_returning_customer_sk#22, ca_state#14, 5), true, [id=#41] - -(53) HashAggregate [codegen id : 18] -Input [3]: [cr_returning_customer_sk#22, ca_state#14, sum#40] -Keys [2]: [cr_returning_customer_sk#22, ca_state#14] -Functions [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_return_amt_inc_tax#24))#42] -Results [2]: [ca_state#14 AS ctr_state#35, MakeDecimal(sum(UnscaledValue(cr_return_amt_inc_tax#24))#42,17,2) AS ctr_total_return#36] - -(54) HashAggregate [codegen id : 18] -Input [2]: [ctr_state#35, ctr_total_return#36] -Keys [1]: [ctr_state#35] -Functions [1]: [partial_avg(ctr_total_return#36)] -Aggregate Attributes [2]: [sum#43, count#44] -Results [3]: [ctr_state#35, sum#45, count#46] - -(55) Exchange -Input [3]: [ctr_state#35, sum#45, count#46] -Arguments: hashpartitioning(ctr_state#35, 5), true, [id=#47] - -(56) HashAggregate [codegen id : 19] -Input [3]: [ctr_state#35, sum#45, count#46] -Keys [1]: [ctr_state#35] -Functions [1]: [avg(ctr_total_return#36)] -Aggregate Attributes [1]: [avg(ctr_total_return#36)#48] -Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#36)#48) * 1.200000), DecimalType(24,7), true) AS (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35 AS ctr_state#35#50] - -(57) Filter [codegen id : 19] -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Condition : isnotnull((CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - -(58) BroadcastExchange -Input [2]: [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] -Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [id=#51] - -(59) BroadcastHashJoin [codegen id : 20] -Left keys [1]: [ctr_state#35] -Right keys [1]: [ctr_state#35#50] -Join condition: (cast(ctr_total_return#36 as decimal(24,7)) > (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49) - (60) Project [codegen id : 20] -Output [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] -Input [19]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_state#35, ctr_total_return#36, (CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))#49, ctr_state#35#50] +Output [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Input [18]: [ctr_customer_sk#16, ctr_total_return#18, c_customer_sk#34, c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49] (61) TakeOrderedAndProject -Input [16]: [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] -Arguments: 100, [c_customer_id#2 ASC NULLS FIRST, c_salutation#4 ASC NULLS FIRST, c_first_name#5 ASC NULLS FIRST, c_last_name#6 ASC NULLS FIRST, ca_street_number#8 ASC NULLS FIRST, ca_street_name#9 ASC NULLS FIRST, ca_street_type#10 ASC NULLS FIRST, ca_suite_number#11 ASC NULLS FIRST, ca_city#12 ASC NULLS FIRST, ca_county#13 ASC NULLS FIRST, ca_state#14 ASC NULLS FIRST, ca_zip#15 ASC NULLS FIRST, ca_country#16 ASC NULLS FIRST, ca_gmt_offset#17 ASC NULLS FIRST, ca_location_type#18 ASC NULLS FIRST, ctr_total_return#36 ASC NULLS FIRST], [c_customer_id#2, c_salutation#4, c_first_name#5, c_last_name#6, ca_street_number#8, ca_street_name#9, ca_street_type#10, ca_suite_number#11, ca_city#12, ca_county#13, ca_state#14, ca_zip#15, ca_country#16, ca_gmt_offset#17, ca_location_type#18, ctr_total_return#36] +Input [16]: [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] +Arguments: 100, [c_customer_id#35 ASC NULLS FIRST, c_salutation#37 ASC NULLS FIRST, c_first_name#38 ASC NULLS FIRST, c_last_name#39 ASC NULLS FIRST, ca_street_number#40 ASC NULLS FIRST, ca_street_name#41 ASC NULLS FIRST, ca_street_type#42 ASC NULLS FIRST, ca_suite_number#43 ASC NULLS FIRST, ca_city#44 ASC NULLS FIRST, ca_county#45 ASC NULLS FIRST, ca_state#10 ASC NULLS FIRST, ca_zip#46 ASC NULLS FIRST, ca_country#47 ASC NULLS FIRST, ca_gmt_offset#48 ASC NULLS FIRST, ca_location_type#49 ASC NULLS FIRST, ctr_total_return#18 ASC NULLS FIRST], [c_customer_id#35, c_salutation#37, c_first_name#38, c_last_name#39, ca_street_number#40, ca_street_name#41, ca_street_type#42, ca_suite_number#43, ca_city#44, ca_county#45, ca_state#10, ca_zip#46, ca_country#47, ca_gmt_offset#48, ca_location_type#49, ctr_total_return#18] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt index c603ab5194..99677b6e39 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/simplified.txt @@ -1,48 +1,29 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] WholeStageCodegen (20) Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_total_return] - BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - Project [c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type,ctr_state,ctr_total_return] - SortMergeJoin [c_customer_sk,ctr_customer_sk] - InputAdapter - WholeStageCodegen (3) - Sort [c_customer_sk] - InputAdapter - Exchange [c_customer_sk] #1 - WholeStageCodegen (2) - Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Filter [c_customer_sk,c_current_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] - InputAdapter - BroadcastExchange #2 - WholeStageCodegen (1) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] - InputAdapter - WholeStageCodegen (11) - Sort [ctr_customer_sk] - InputAdapter - Exchange [ctr_customer_sk] #3 - WholeStageCodegen (10) + SortMergeJoin [ctr_customer_sk,c_customer_sk] + InputAdapter + WholeStageCodegen (16) + Sort [ctr_customer_sk] + InputAdapter + Exchange [ctr_customer_sk] #1 + WholeStageCodegen (15) + Project [ctr_customer_sk,ctr_total_return] + BroadcastHashJoin [ctr_state,ctr_state,ctr_total_return,(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] Filter [ctr_total_return] HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_customer_sk,ctr_state,ctr_total_return,sum] InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #4 - WholeStageCodegen (9) + Exchange [cr_returning_customer_sk,ca_state] #2 + WholeStageCodegen (6) HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] SortMergeJoin [cr_returning_addr_sk,ca_address_sk] InputAdapter - WholeStageCodegen (6) + WholeStageCodegen (3) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #5 - WholeStageCodegen (5) + Exchange [cr_returning_addr_sk] #3 + WholeStageCodegen (2) Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] BroadcastHashJoin [cr_returned_date_sk,d_date_sk] Filter [cr_returned_date_sk,cr_returning_addr_sk,cr_returning_customer_sk] @@ -50,55 +31,74 @@ TakeOrderedAndProject [c_customer_id,c_salutation,c_first_name,c_last_name,ca_st InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (4) + BroadcastExchange #4 + WholeStageCodegen (1) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (8) + WholeStageCodegen (5) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #7 - WholeStageCodegen (7) + Exchange [ca_address_sk] #5 + WholeStageCodegen (4) Filter [ca_address_sk,ca_state] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (14) + Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] + HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] + InputAdapter + Exchange [ctr_state] #7 + WholeStageCodegen (13) + HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] + HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + InputAdapter + Exchange [cr_returning_customer_sk,ca_state] #8 + WholeStageCodegen (12) + HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] + Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] + SortMergeJoin [cr_returning_addr_sk,ca_address_sk] + InputAdapter + WholeStageCodegen (9) + Sort [cr_returning_addr_sk] + InputAdapter + Exchange [cr_returning_addr_sk] #9 + WholeStageCodegen (8) + Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_returned_date_sk,cr_returning_addr_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] + InputAdapter + ReusedExchange [d_date_sk] #4 + InputAdapter + WholeStageCodegen (11) + Sort [ca_address_sk] + InputAdapter + ReusedExchange [ca_address_sk,ca_state] #5 InputAdapter - BroadcastExchange #8 - WholeStageCodegen (19) - Filter [(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6)))] - HashAggregate [ctr_state,sum,count] [avg(ctr_total_return),(CAST(avg(ctr_total_return) AS DECIMAL(21,6)) * CAST(1.2 AS DECIMAL(21,6))),ctr_state,sum,count] - InputAdapter - Exchange [ctr_state] #9 - WholeStageCodegen (18) - HashAggregate [ctr_state,ctr_total_return] [sum,count,sum,count] - HashAggregate [cr_returning_customer_sk,ca_state,sum] [sum(UnscaledValue(cr_return_amt_inc_tax)),ctr_state,ctr_total_return,sum] + WholeStageCodegen (19) + Sort [c_customer_sk] + InputAdapter + Exchange [c_customer_sk] #10 + WholeStageCodegen (18) + Project [c_customer_sk,c_customer_id,c_salutation,c_first_name,c_last_name,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Filter [c_customer_sk,c_current_addr_sk] + ColumnarToRow InputAdapter - Exchange [cr_returning_customer_sk,ca_state] #10 - WholeStageCodegen (17) - HashAggregate [cr_returning_customer_sk,ca_state,cr_return_amt_inc_tax] [sum,sum] - Project [cr_returning_customer_sk,cr_return_amt_inc_tax,ca_state] - SortMergeJoin [cr_returning_addr_sk,ca_address_sk] - InputAdapter - WholeStageCodegen (14) - Sort [cr_returning_addr_sk] - InputAdapter - Exchange [cr_returning_addr_sk] #11 - WholeStageCodegen (13) - Project [cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_returned_date_sk,cr_returning_addr_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_returning_addr_sk,cr_return_amt_inc_tax] - InputAdapter - ReusedExchange [d_date_sk] #6 - InputAdapter - WholeStageCodegen (16) - Sort [ca_address_sk] - InputAdapter - ReusedExchange [ca_address_sk,ca_state] #7 + Scan parquet default.customer [c_customer_sk,c_customer_id,c_current_addr_sk,c_salutation,c_first_name,c_last_name] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (17) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_street_number,ca_street_name,ca_street_type,ca_suite_number,ca_city,ca_county,ca_state,ca_zip,ca_country,ca_gmt_offset,ca_location_type] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt index 4e85516b59..6bcbe470ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/explain.txt @@ -8,206 +8,206 @@ +- * BroadcastHashJoin Inner BuildRight (41) :- * Project (36) : +- * BroadcastHashJoin Inner BuildRight (35) - : :- * Project (23) - : : +- * BroadcastHashJoin Inner BuildRight (22) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) - : : : :- * Project (9) - : : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : : :- * Filter (3) - : : : : : +- * ColumnarToRow (2) - : : : : : +- Scan parquet default.customer (1) - : : : : +- BroadcastExchange (7) - : : : : +- * Filter (6) - : : : : +- * ColumnarToRow (5) - : : : : +- Scan parquet default.customer_demographics (4) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.household_demographics (10) - : : +- BroadcastExchange (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.customer_address (17) + : :- * Project (30) + : : +- * BroadcastHashJoin Inner BuildRight (29) + : : :- * Project (23) + : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : :- * Project (16) + : : : : +- * BroadcastHashJoin Inner BuildLeft (15) + : : : : :- BroadcastExchange (11) + : : : : : +- * Project (10) + : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : :- * Filter (3) + : : : : : : +- * ColumnarToRow (2) + : : : : : : +- Scan parquet default.catalog_returns (1) + : : : : : +- BroadcastExchange (8) + : : : : : +- * Project (7) + : : : : : +- * Filter (6) + : : : : : +- * ColumnarToRow (5) + : : : : : +- Scan parquet default.date_dim (4) + : : : : +- * Filter (14) + : : : : +- * ColumnarToRow (13) + : : : : +- Scan parquet default.customer (12) + : : : +- BroadcastExchange (21) + : : : +- * Project (20) + : : : +- * Filter (19) + : : : +- * ColumnarToRow (18) + : : : +- Scan parquet default.household_demographics (17) + : : +- BroadcastExchange (28) + : : +- * Project (27) + : : +- * Filter (26) + : : +- * ColumnarToRow (25) + : : +- Scan parquet default.customer_address (24) : +- BroadcastExchange (34) - : +- * Project (33) - : +- * BroadcastHashJoin Inner BuildRight (32) - : :- * Filter (26) - : : +- * ColumnarToRow (25) - : : +- Scan parquet default.catalog_returns (24) - : +- BroadcastExchange (31) - : +- * Project (30) - : +- * Filter (29) - : +- * ColumnarToRow (28) - : +- Scan parquet default.date_dim (27) + : +- * Filter (33) + : +- * ColumnarToRow (32) + : +- Scan parquet default.customer_demographics (31) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(1) Scan parquet default.customer -Output [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(1) Scan parquet default.catalog_returns +Output [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer] -PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] +(2) ColumnarToRow [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] -(3) Filter [codegen id : 7] -Input [4]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4] -Condition : (((isnotnull(c_customer_sk#1) AND isnotnull(c_current_addr_sk#4)) AND isnotnull(c_current_cdemo_sk#2)) AND isnotnull(c_current_hdemo_sk#3)) +(3) Filter [codegen id : 2] +Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Condition : ((isnotnull(cr_call_center_sk#3) AND isnotnull(cr_returned_date_sk#1)) AND isnotnull(cr_returning_customer_sk#2)) -(4) Scan parquet default.customer_demographics -Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(4) Scan parquet default.date_dim +Output [3]: [d_date_sk#5, d_year#6, d_moy#7] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] (6) Filter [codegen id : 1] -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Condition : ((((cd_marital_status#6 = M) AND (cd_education_status#7 = Unknown)) OR ((cd_marital_status#6 = W) AND (cd_education_status#7 = Advanced Degree))) AND isnotnull(cd_demo_sk#5)) +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Condition : ((((isnotnull(d_year#6) AND isnotnull(d_moy#7)) AND (d_year#6 = 1998)) AND (d_moy#7 = 11)) AND isnotnull(d_date_sk#5)) + +(7) Project [codegen id : 1] +Output [1]: [d_date_sk#5] +Input [3]: [d_date_sk#5, d_year#6, d_moy#7] -(7) BroadcastExchange -Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#8] +(8) BroadcastExchange +Input [1]: [d_date_sk#5] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] -(8) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#2] -Right keys [1]: [cd_demo_sk#5] +(9) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [cr_returned_date_sk#1] +Right keys [1]: [d_date_sk#5] Join condition: None -(9) Project [codegen id : 7] -Output [5]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [7]: [c_customer_sk#1, c_current_cdemo_sk#2, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] +(10) Project [codegen id : 2] +Output [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Input [5]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, d_date_sk#5] -(10) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#9, hd_buy_potential#10] -Batched: true -Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] -ReadSchema: struct +(11) BroadcastExchange +Input [3]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] -(11) ColumnarToRow [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] - -(12) Filter [codegen id : 2] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND StartsWith(hd_buy_potential#10, Unknown)) AND isnotnull(hd_demo_sk#9)) +(12) Scan parquet default.customer +Output [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(13) Project [codegen id : 2] -Output [1]: [hd_demo_sk#9] -Input [2]: [hd_demo_sk#9, hd_buy_potential#10] +(13) ColumnarToRow +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(14) BroadcastExchange -Input [1]: [hd_demo_sk#9] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] +(14) Filter +Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Condition : (((isnotnull(c_customer_sk#10) AND isnotnull(c_current_addr_sk#13)) AND isnotnull(c_current_cdemo_sk#11)) AND isnotnull(c_current_hdemo_sk#12)) (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#3] -Right keys [1]: [hd_demo_sk#9] +Left keys [1]: [cr_returning_customer_sk#2] +Right keys [1]: [c_customer_sk#10] Join condition: None (16) Project [codegen id : 7] -Output [4]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7] -Input [6]: [c_customer_sk#1, c_current_hdemo_sk#3, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, hd_demo_sk#9] +Output [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +Input [7]: [cr_returning_customer_sk#2, cr_call_center_sk#3, cr_net_loss#4, c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] -(17) Scan parquet default.customer_address -Output [2]: [ca_address_sk#12, ca_gmt_offset#13] +(17) Scan parquet default.household_demographics +Output [2]: [hd_demo_sk#14, hd_buy_potential#15] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct (18) ColumnarToRow [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (19) Filter [codegen id : 3] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] -Condition : ((isnotnull(ca_gmt_offset#13) AND (ca_gmt_offset#13 = -7.00)) AND isnotnull(ca_address_sk#12)) +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Condition : ((isnotnull(hd_buy_potential#15) AND StartsWith(hd_buy_potential#15, Unknown)) AND isnotnull(hd_demo_sk#14)) (20) Project [codegen id : 3] -Output [1]: [ca_address_sk#12] -Input [2]: [ca_address_sk#12, ca_gmt_offset#13] +Output [1]: [hd_demo_sk#14] +Input [2]: [hd_demo_sk#14, hd_buy_potential#15] (21) BroadcastExchange -Input [1]: [ca_address_sk#12] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [1]: [hd_demo_sk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#4] -Right keys [1]: [ca_address_sk#12] +Left keys [1]: [c_current_hdemo_sk#12] +Right keys [1]: [hd_demo_sk#14] Join condition: None (23) Project [codegen id : 7] -Output [3]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7] -Input [5]: [c_customer_sk#1, c_current_addr_sk#4, cd_marital_status#6, cd_education_status#7, ca_address_sk#12] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13, hd_demo_sk#14] -(24) Scan parquet default.catalog_returns -Output [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(24) Scan parquet default.customer_address +Output [2]: [ca_address_sk#17, ca_gmt_offset#18] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_returns] -PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_gmt_offset), EqualTo(ca_gmt_offset,-7.00), IsNotNull(ca_address_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +(25) ColumnarToRow [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(26) Filter [codegen id : 5] -Input [4]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Condition : ((isnotnull(cr_call_center_sk#17) AND isnotnull(cr_returned_date_sk#15)) AND isnotnull(cr_returning_customer_sk#16)) +(26) Filter [codegen id : 4] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +Condition : ((isnotnull(ca_gmt_offset#18) AND (ca_gmt_offset#18 = -7.00)) AND isnotnull(ca_address_sk#17)) -(27) Scan parquet default.date_dim -Output [3]: [d_date_sk#19, d_year#20, d_moy#21] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,1998), EqualTo(d_moy,11), IsNotNull(d_date_sk)] -ReadSchema: struct +(27) Project [codegen id : 4] +Output [1]: [ca_address_sk#17] +Input [2]: [ca_address_sk#17, ca_gmt_offset#18] -(28) ColumnarToRow [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(28) BroadcastExchange +Input [1]: [ca_address_sk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] -(29) Filter [codegen id : 4] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -Condition : ((((isnotnull(d_year#20) AND isnotnull(d_moy#21)) AND (d_year#20 = 1998)) AND (d_moy#21 = 11)) AND isnotnull(d_date_sk#19)) +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#13] +Right keys [1]: [ca_address_sk#17] +Join condition: None -(30) Project [codegen id : 4] -Output [1]: [d_date_sk#19] -Input [3]: [d_date_sk#19, d_year#20, d_moy#21] +(30) Project [codegen id : 7] +Output [3]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11] +Input [5]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, c_current_addr_sk#13, ca_address_sk#17] -(31) BroadcastExchange -Input [1]: [d_date_sk#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] +(31) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_demographics] +PushedFilters: [Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Unknown)),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,Advanced Degree))), IsNotNull(cd_demo_sk)] +ReadSchema: struct -(32) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [cr_returned_date_sk#15] -Right keys [1]: [d_date_sk#19] -Join condition: None +(32) ColumnarToRow [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] -(33) Project [codegen id : 5] -Output [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Input [5]: [cr_returned_date_sk#15, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18, d_date_sk#19] +(33) Filter [codegen id : 5] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Condition : ((((cd_marital_status#21 = M) AND (cd_education_status#22 = Unknown)) OR ((cd_marital_status#21 = W) AND (cd_education_status#22 = Advanced Degree))) AND isnotnull(cd_demo_sk#20)) (34) BroadcastExchange -Input [3]: [cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] +Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_customer_sk#1] -Right keys [1]: [cr_returning_customer_sk#16] +Left keys [1]: [c_current_cdemo_sk#11] +Right keys [1]: [cd_demo_sk#20] Join condition: None (36) Project [codegen id : 7] -Output [4]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18] -Input [6]: [c_customer_sk#1, cd_marital_status#6, cd_education_status#7, cr_returning_customer_sk#16, cr_call_center_sk#17, cr_net_loss#18] +Output [4]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [6]: [cr_call_center_sk#3, cr_net_loss#4, c_current_cdemo_sk#11, cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] (37) Scan parquet default.call_center Output [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] @@ -228,35 +228,35 @@ Input [4]: [cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#2 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#28] (41) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_call_center_sk#17] +Left keys [1]: [cr_call_center_sk#3] Right keys [1]: [cc_call_center_sk#24] Join condition: None (42) Project [codegen id : 7] -Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Input [8]: [cd_marital_status#6, cd_education_status#7, cr_call_center_sk#17, cr_net_loss#18, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] +Output [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Input [8]: [cr_call_center_sk#3, cr_net_loss#4, cd_marital_status#21, cd_education_status#22, cc_call_center_sk#24, cc_call_center_id#25, cc_name#26, cc_manager#27] (43) HashAggregate [codegen id : 7] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#18, cd_marital_status#6, cd_education_status#7] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#18))] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cr_net_loss#4, cd_marital_status#21, cd_education_status#22] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [partial_sum(UnscaledValue(cr_net_loss#4))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] (44) Exchange -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, 5), true, [id=#31] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Arguments: hashpartitioning(cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, 5), ENSURE_REQUIREMENTS, [id=#31] (45) HashAggregate [codegen id : 8] -Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] -Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7] -Functions [1]: [sum(UnscaledValue(cr_net_loss#18))] -Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#18))#32] -Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#18))#32,17,2) AS Returns_Loss#36] +Input [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Keys [5]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22] +Functions [1]: [sum(UnscaledValue(cr_net_loss#4))] +Aggregate Attributes [1]: [sum(UnscaledValue(cr_net_loss#4))#32] +Results [4]: [cc_call_center_id#25 AS Call_Center#33, cc_name#26 AS Call_Center_Name#34, cc_manager#27 AS Manager#35, MakeDecimal(sum(UnscaledValue(cr_net_loss#4))#32,17,2) AS Returns_Loss#36] (46) Exchange Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] -Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), true, [id=#37] +Arguments: rangepartitioning(Returns_Loss#36 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#37] (47) Sort [codegen id : 9] Input [4]: [Call_Center#33, Call_Center_Name#34, Manager#35, Returns_Loss#36] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt index 87beb3b565..6c8d629fee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q91.sf100/simplified.txt @@ -10,58 +10,58 @@ WholeStageCodegen (9) HashAggregate [cc_call_center_id,cc_name,cc_manager,cd_marital_status,cd_education_status,cr_net_loss] [sum,sum] Project [cc_call_center_id,cc_name,cc_manager,cr_net_loss,cd_marital_status,cd_education_status] BroadcastHashJoin [cr_call_center_sk,cc_call_center_sk] - Project [cd_marital_status,cd_education_status,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [c_customer_sk,cr_returning_customer_sk] - Project [c_customer_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,cd_marital_status,cd_education_status] + BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] BroadcastHashJoin [c_current_hdemo_sk,hd_demo_sk] - Project [c_customer_sk,c_current_hdemo_sk,c_current_addr_sk,cd_marital_status,cd_education_status] - BroadcastHashJoin [c_current_cdemo_sk,cd_demo_sk] + Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] + BroadcastHashJoin [cr_returning_customer_sk,c_customer_sk] + InputAdapter + BroadcastExchange #3 + WholeStageCodegen (2) + Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + BroadcastHashJoin [cr_returned_date_sk,d_date_sk] + Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (1) + Project [d_date_sk] + Filter [d_year,d_moy,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_year,d_moy] Filter [c_customer_sk,c_current_addr_sk,c_current_cdemo_sk,c_current_hdemo_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_hdemo_sk,c_current_addr_sk] - InputAdapter - BroadcastExchange #3 - WholeStageCodegen (1) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter - BroadcastExchange #4 - WholeStageCodegen (2) + BroadcastExchange #5 + WholeStageCodegen (3) Project [hd_demo_sk] Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) + BroadcastExchange #6 + WholeStageCodegen (4) Project [ca_address_sk] Filter [ca_gmt_offset,ca_address_sk] ColumnarToRow InputAdapter Scan parquet default.customer_address [ca_address_sk,ca_gmt_offset] InputAdapter - BroadcastExchange #6 + BroadcastExchange #7 WholeStageCodegen (5) - Project [cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] - BroadcastHashJoin [cr_returned_date_sk,d_date_sk] - Filter [cr_call_center_sk,cr_returned_date_sk,cr_returning_customer_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_returns [cr_returned_date_sk,cr_returning_customer_sk,cr_call_center_sk,cr_net_loss] + Filter [cd_marital_status,cd_education_status,cd_demo_sk] + ColumnarToRow InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_moy,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year,d_moy] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt index 2d76deefca..f6c5258701 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/explain.txt @@ -34,24 +34,24 @@ TakeOrderedAndProject (160) : +- * Sort (46) : +- Exchange (45) : +- * Project (44) - : +- * SortMergeJoin Inner (43) - : :- * Sort (37) - : : +- Exchange (36) - : : +- * Project (35) - : : +- * BroadcastHashJoin Inner BuildRight (34) - : : :- * Project (29) - : : : +- * Filter (28) - : : : +- * ColumnarToRow (27) - : : : +- Scan parquet default.customer (26) - : : +- BroadcastExchange (33) - : : +- * Filter (32) - : : +- * ColumnarToRow (31) - : : +- Scan parquet default.customer_address (30) - : +- * Sort (42) - : +- Exchange (41) - : +- * Filter (40) - : +- * ColumnarToRow (39) - : +- Scan parquet default.customer_demographics (38) + : +- * BroadcastHashJoin Inner BuildRight (43) + : :- * Project (38) + : : +- * SortMergeJoin Inner (37) + : : :- * Sort (31) + : : : +- Exchange (30) + : : : +- * Project (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.customer (26) + : : +- * Sort (36) + : : +- Exchange (35) + : : +- * Filter (34) + : : +- * ColumnarToRow (33) + : : +- Scan parquet default.customer_demographics (32) + : +- BroadcastExchange (42) + : +- * Filter (41) + : +- * ColumnarToRow (40) + : +- Scan parquet default.customer_address (39) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -266,7 +266,7 @@ Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6 (24) Exchange Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] -Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), true, [id=#21] +Arguments: hashpartitioning(cs_bill_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#21] (25) Sort [codegen id : 5] Input [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -279,89 +279,89 @@ Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [In(c_birth_month, [9,5,12,4,1,10]), IsNotNull(c_customer_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(27) ColumnarToRow [codegen id : 7] +(27) ColumnarToRow [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(28) Filter [codegen id : 7] +(28) Filter [codegen id : 6] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] Condition : (((c_birth_month#25 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#22)) AND isnotnull(c_current_cdemo_sk#23)) AND isnotnull(c_current_addr_sk#24)) -(29) Project [codegen id : 7] +(29) Project [codegen id : 6] Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] -(30) Scan parquet default.customer_address -Output [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] -ReadSchema: struct - -(31) ColumnarToRow [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(32) Filter [codegen id : 6] -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) - -(33) BroadcastExchange -Input [4]: [ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#31] - -(34) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] -Join condition: None - -(35) Project [codegen id : 7] -Output [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [8]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_county#28, ca_state#29, ca_country#30] - -(36) Exchange -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#32] +(30) Exchange +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#27] -(37) Sort [codegen id : 8] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +(31) Sort [codegen id : 7] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(38) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#33] +(32) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 9] -Input [1]: [cd_demo_sk#33] +(33) ColumnarToRow [codegen id : 8] +Input [1]: [cd_demo_sk#28] -(40) Filter [codegen id : 9] -Input [1]: [cd_demo_sk#33] -Condition : isnotnull(cd_demo_sk#33) +(34) Filter [codegen id : 8] +Input [1]: [cd_demo_sk#28] +Condition : isnotnull(cd_demo_sk#28) -(41) Exchange -Input [1]: [cd_demo_sk#33] -Arguments: hashpartitioning(cd_demo_sk#33, 5), true, [id=#34] +(35) Exchange +Input [1]: [cd_demo_sk#28] +Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] -(42) Sort [codegen id : 10] -Input [1]: [cd_demo_sk#33] -Arguments: [cd_demo_sk#33 ASC NULLS FIRST], false, 0 +(36) Sort [codegen id : 9] +Input [1]: [cd_demo_sk#28] +Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 -(43) SortMergeJoin [codegen id : 11] +(37) SortMergeJoin [codegen id : 11] Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#33] +Right keys [1]: [cd_demo_sk#28] +Join condition: None + +(38) Project [codegen id : 11] +Output [3]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, cd_demo_sk#28] + +(39) Scan parquet default.customer_address +Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Batched: true +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] +ReadSchema: struct + +(40) ColumnarToRow [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] + +(41) Filter [codegen id : 10] +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) + +(42) BroadcastExchange +Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#34] + +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [c_current_addr_sk#24] +Right keys [1]: [ca_address_sk#30] Join condition: None (44) Project [codegen id : 11] -Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30, cd_demo_sk#33] +Output [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] (45) Exchange -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#35] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#35] (46) Sort [codegen id : 12] -Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] @@ -370,26 +370,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (48) Project [codegen id : 13] -Output [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#28, ca_state#29, ca_country#30] +Output [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [13]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_county#31, ca_state#32, ca_country#33] (49) HashAggregate [codegen id : 13] -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54, sum#55, count#56] -Results [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Results [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] (50) Exchange -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, ca_county#28, 5), true, [id=#71] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, ca_county#31, 5), ENSURE_REQUIREMENTS, [id=#71] (51) HashAggregate [codegen id : 14] -Input [18]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] -Keys [4]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28] +Input [18]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, sum#57, count#58, sum#59, count#60, sum#61, count#62, sum#63, count#64, sum#65, count#66, sum#67, count#68, sum#69, count#70] +Keys [4]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#72, avg(agg2#37)#73, avg(agg3#38)#74, avg(agg4#39)#75, avg(agg5#40)#76, avg(agg6#41)#77, avg(agg7#42)#78] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, avg(agg1#36)#72 AS agg1#79, avg(agg2#37)#73 AS agg2#80, avg(agg3#38)#74 AS agg3#81, avg(agg4#39)#75 AS agg4#82, avg(agg5#40)#76 AS agg5#83, avg(agg6#41)#77 AS agg6#84, avg(agg7#42)#78 AS agg7#85] (52) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -417,41 +417,41 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (59) ColumnarToRow [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (60) Filter [codegen id : 20] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (61) BroadcastExchange -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#86] (62) BroadcastHashJoin [codegen id : 21] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (63) Project [codegen id : 21] -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_state#29, ca_country#30] +Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Input [7]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_state#32, ca_country#33] (64) Exchange -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#87] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#87] (65) Sort [codegen id : 22] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 41] +(66) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#88] (67) Sort [codegen id : 24] @@ -464,15 +464,15 @@ Right keys [1]: [cd_demo_sk#88] Join condition: None (69) Project [codegen id : 25] -Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#29, ca_country#30, cd_demo_sk#88] +Output [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_state#32, ca_country#33, cd_demo_sk#88] (70) Exchange -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#89] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#89] (71) Sort [codegen id : 26] -Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Input [4]: [c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] @@ -481,26 +481,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (73) Project [codegen id : 27] -Output [10]: [i_item_id#19, ca_country#30, ca_state#29, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#29, ca_country#30] +Output [10]: [i_item_id#19, ca_country#33, ca_state#32, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [12]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_state#32, ca_country#33] (74) HashAggregate [codegen id : 27] -Input [10]: [i_item_id#19, ca_country#30, ca_state#29, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [10]: [i_item_id#19, ca_country#33, ca_state#32, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#90, count#91, sum#92, count#93, sum#94, count#95, sum#96, count#97, sum#98, count#99, sum#100, count#101, sum#102, count#103] -Results [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Results [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] (75) Exchange -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, ca_state#29, 5), true, [id=#118] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, ca_state#32, 5), ENSURE_REQUIREMENTS, [id=#118] (76) HashAggregate [codegen id : 28] -Input [17]: [i_item_id#19, ca_country#30, ca_state#29, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] -Keys [3]: [i_item_id#19, ca_country#30, ca_state#29] +Input [17]: [i_item_id#19, ca_country#33, ca_state#32, sum#104, count#105, sum#106, count#107, sum#108, count#109, sum#110, count#111, sum#112, count#113, sum#114, count#115, sum#116, count#117] +Keys [3]: [i_item_id#19, ca_country#33, ca_state#32] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#119, avg(agg2#37)#120, avg(agg3#38)#121, avg(agg4#39)#122, avg(agg5#40)#123, avg(agg6#41)#124, avg(agg7#42)#125] -Results [11]: [i_item_id#19, ca_country#30, ca_state#29, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] +Results [11]: [i_item_id#19, ca_country#33, ca_state#32, null AS county#126, avg(agg1#36)#119 AS agg1#127, avg(agg2#37)#120 AS agg2#128, avg(agg3#38)#121 AS agg3#129, avg(agg4#39)#122 AS agg4#130, avg(agg5#40)#123 AS agg5#131, avg(agg6#41)#124 AS agg6#132, avg(agg7#42)#125 AS agg7#133] (77) ReusedExchange [Reuses operator id: 24] Output [8]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19] @@ -528,45 +528,45 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (84) ColumnarToRow [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (85) Filter [codegen id : 34] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#27, ca_country#30] -Input [3]: [ca_address_sk#27, ca_state#29, ca_country#30] +Output [2]: [ca_address_sk#30, ca_country#33] +Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] (87) BroadcastExchange -Input [2]: [ca_address_sk#27, ca_country#30] +Input [2]: [ca_address_sk#30, ca_country#33] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] (88) BroadcastHashJoin [codegen id : 35] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (89) Project [codegen id : 35] -Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27, ca_country#30] +Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Input [6]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30, ca_country#33] (90) Exchange -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), true, [id=#135] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_current_cdemo_sk#23, 5), ENSURE_REQUIREMENTS, [id=#135] (91) Sort [codegen id : 36] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30] +Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33] Arguments: [c_current_cdemo_sk#23 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 41] +(92) ReusedExchange [Reuses operator id: 35] Output [1]: [cd_demo_sk#136] (93) Sort [codegen id : 38] @@ -579,15 +579,15 @@ Right keys [1]: [cd_demo_sk#136] Join condition: None (95) Project [codegen id : 39] -Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#30, cd_demo_sk#136] +Output [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, ca_country#33, cd_demo_sk#136] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] -Arguments: hashpartitioning(c_customer_sk#22, 5), true, [id=#137] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] +Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#30] +Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] @@ -596,26 +596,26 @@ Right keys [1]: [c_customer_sk#22] Join condition: None (99) Project [codegen id : 41] -Output [9]: [i_item_id#19, ca_country#30, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] -Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#30] +Output [9]: [i_item_id#19, ca_country#33, cast(cs_quantity#5 as decimal(12,2)) AS agg1#36, cast(cs_list_price#6 as decimal(12,2)) AS agg2#37, cast(cs_coupon_amt#8 as decimal(12,2)) AS agg3#38, cast(cs_sales_price#7 as decimal(12,2)) AS agg4#39, cast(cs_net_profit#9 as decimal(12,2)) AS agg5#40, cast(c_birth_year#26 as decimal(12,2)) AS agg6#41, cast(cd_dep_count#13 as decimal(12,2)) AS agg7#42] +Input [11]: [cs_bill_customer_sk#2, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, i_item_id#19, c_customer_sk#22, c_birth_year#26, ca_country#33] (100) HashAggregate [codegen id : 41] -Input [9]: [i_item_id#19, ca_country#30, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [2]: [i_item_id#19, ca_country#30] +Input [9]: [i_item_id#19, ca_country#33, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [partial_avg(agg1#36), partial_avg(agg2#37), partial_avg(agg3#38), partial_avg(agg4#39), partial_avg(agg5#40), partial_avg(agg6#41), partial_avg(agg7#42)] Aggregate Attributes [14]: [sum#138, count#139, sum#140, count#141, sum#142, count#143, sum#144, count#145, sum#146, count#147, sum#148, count#149, sum#150, count#151] -Results [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Results [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] (101) Exchange -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Arguments: hashpartitioning(i_item_id#19, ca_country#30, 5), true, [id=#166] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Arguments: hashpartitioning(i_item_id#19, ca_country#33, 5), ENSURE_REQUIREMENTS, [id=#166] (102) HashAggregate [codegen id : 42] -Input [16]: [i_item_id#19, ca_country#30, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] -Keys [2]: [i_item_id#19, ca_country#30] +Input [16]: [i_item_id#19, ca_country#33, sum#152, count#153, sum#154, count#155, sum#156, count#157, sum#158, count#159, sum#160, count#161, sum#162, count#163, sum#164, count#165] +Keys [2]: [i_item_id#19, ca_country#33] Functions [7]: [avg(agg1#36), avg(agg2#37), avg(agg3#38), avg(agg4#39), avg(agg5#40), avg(agg6#41), avg(agg7#42)] Aggregate Attributes [7]: [avg(agg1#36)#167, avg(agg2#37)#168, avg(agg3#38)#169, avg(agg4#39)#170, avg(agg5#40)#171, avg(agg6#41)#172, avg(agg7#42)#173] -Results [11]: [i_item_id#19, ca_country#30, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] +Results [11]: [i_item_id#19, ca_country#33, null AS ca_state#174, null AS county#175, avg(agg1#36)#167 AS agg1#176, avg(agg2#37)#168 AS agg2#177, avg(agg3#38)#169 AS agg3#178, avg(agg4#39)#170 AS agg4#179, avg(agg5#40)#171 AS agg5#180, avg(agg6#41)#172 AS agg6#181, avg(agg7#42)#173 AS agg7#182] (103) Scan parquet default.catalog_sales Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_bill_cdemo_sk#3, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9] @@ -674,35 +674,35 @@ Output [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_bi Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#27, ca_state#29] +Output [2]: [ca_address_sk#30, ca_state#32] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [In(ca_state, [ND,WI,AL,NC,OK,MS,TN]), IsNotNull(ca_address_sk)] ReadSchema: struct (117) ColumnarToRow [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] +Input [2]: [ca_address_sk#30, ca_state#32] (118) Filter [codegen id : 45] -Input [2]: [ca_address_sk#27, ca_state#29] -Condition : (ca_state#29 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#27)) +Input [2]: [ca_address_sk#30, ca_state#32] +Condition : (ca_state#32 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#30)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#27] -Input [2]: [ca_address_sk#27, ca_state#29] +Output [1]: [ca_address_sk#30] +Input [2]: [ca_address_sk#30, ca_state#32] (120) BroadcastExchange -Input [1]: [ca_address_sk#27] +Input [1]: [ca_address_sk#30] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] (121) BroadcastHashJoin [codegen id : 46] Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#27] +Right keys [1]: [ca_address_sk#30] Join condition: None (122) Project [codegen id : 46] Output [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#27] +Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_year#26, ca_address_sk#30] (123) BroadcastExchange Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] @@ -765,7 +765,7 @@ Results [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, co (136) Exchange Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] -Arguments: hashpartitioning(i_item_id#19, 5), true, [id=#215] +Arguments: hashpartitioning(i_item_id#19, 5), ENSURE_REQUIREMENTS, [id=#215] (137) HashAggregate [codegen id : 50] Input [15]: [i_item_id#19, sum#201, count#202, sum#203, count#204, sum#205, count#206, sum#207, count#208, sum#209, count#210, sum#211, count#212, sum#213, count#214] @@ -860,7 +860,7 @@ Results [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#2 (157) Exchange Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] -Arguments: SinglePartition, true, [id=#262] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#262] (158) HashAggregate [codegen id : 58] Input [14]: [sum#248, count#249, sum#250, count#251, sum#252, count#253, sum#254, count#255, sum#256, count#257, sum#258, count#259, sum#260, count#261] @@ -872,6 +872,6 @@ Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#2 (159) Union (160) TakeOrderedAndProject -Input [11]: [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] -Arguments: 100, [ca_country#30 ASC NULLS FIRST, ca_state#29 ASC NULLS FIRST, ca_county#28 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#30, ca_state#29, ca_county#28, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Input [11]: [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] +Arguments: 100, [ca_country#33 ASC NULLS FIRST, ca_state#32 ASC NULLS FIRST, ca_county#31 ASC NULLS FIRST, i_item_id#19 ASC NULLS FIRST], [i_item_id#19, ca_country#33, ca_state#32, ca_county#31, agg1#79, agg2#80, agg3#81, agg4#82, agg5#83, agg6#84, agg7#85] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt index 5514e335f1..4566929712 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q18a.sf100/simplified.txt @@ -54,37 +54,37 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag Exchange [c_customer_sk] #6 WholeStageCodegen (11) Project [c_customer_sk,c_birth_year,ca_county,ca_state,ca_country] - SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] - InputAdapter - WholeStageCodegen (8) - Sort [c_current_cdemo_sk] - InputAdapter - Exchange [c_current_cdemo_sk] #7 - WholeStageCodegen (7) - Project [c_customer_sk,c_current_cdemo_sk,c_birth_year,ca_county,ca_state,ca_country] - BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + BroadcastHashJoin [c_current_addr_sk,ca_address_sk] + Project [c_customer_sk,c_current_addr_sk,c_birth_year] + SortMergeJoin [c_current_cdemo_sk,cd_demo_sk] + InputAdapter + WholeStageCodegen (7) + Sort [c_current_cdemo_sk] + InputAdapter + Exchange [c_current_cdemo_sk] #7 + WholeStageCodegen (6) Project [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_year] Filter [c_birth_month,c_customer_sk,c_current_cdemo_sk,c_current_addr_sk] ColumnarToRow InputAdapter Scan parquet default.customer [c_customer_sk,c_current_cdemo_sk,c_current_addr_sk,c_birth_month,c_birth_year] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (6) - Filter [ca_state,ca_address_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] + InputAdapter + WholeStageCodegen (9) + Sort [cd_demo_sk] + InputAdapter + Exchange [cd_demo_sk] #8 + WholeStageCodegen (8) + Filter [cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk] InputAdapter - WholeStageCodegen (10) - Sort [cd_demo_sk] - InputAdapter - Exchange [cd_demo_sk] #9 - WholeStageCodegen (9) - Filter [cd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk] + BroadcastExchange #9 + WholeStageCodegen (10) + Filter [ca_state,ca_address_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_address [ca_address_sk,ca_county,ca_state,ca_country] WholeStageCodegen (28) HashAggregate [i_item_id,ca_country,ca_state,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -130,7 +130,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (24) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 WholeStageCodegen (42) HashAggregate [i_item_id,ca_country,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter @@ -177,7 +177,7 @@ TakeOrderedAndProject [ca_country,ca_state,ca_county,i_item_id,agg1,agg2,agg3,ag WholeStageCodegen (38) Sort [cd_demo_sk] InputAdapter - ReusedExchange [cd_demo_sk] #9 + ReusedExchange [cd_demo_sk] #8 WholeStageCodegen (50) HashAggregate [i_item_id,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] [avg(agg1),avg(agg2),avg(agg3),avg(agg4),avg(agg5),avg(agg6),avg(agg7),ca_country,ca_state,county,agg1,agg2,agg3,agg4,agg5,agg6,agg7,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt index 432ef4db6b..411cbf4809 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt @@ -13,8 +13,8 @@ TakeOrderedAndProject (94) : : +- * HashAggregate (23) : : +- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) - : : :- * Project (16) - : : : +- * BroadcastHashJoin Inner BuildRight (15) + : : :- * Project (15) + : : : +- * BroadcastHashJoin Inner BuildRight (14) : : : :- Union (9) : : : : :- * Project (4) : : : : : +- * Filter (3) @@ -24,22 +24,22 @@ TakeOrderedAndProject (94) : : : : +- * Filter (7) : : : : +- * ColumnarToRow (6) : : : : +- Scan parquet default.store_returns (5) - : : : +- BroadcastExchange (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- * ColumnarToRow (11) - : : : +- Scan parquet default.date_dim (10) + : : : +- BroadcastExchange (13) + : : : +- * Filter (12) + : : : +- * ColumnarToRow (11) + : : : +- Scan parquet default.store (10) : : +- BroadcastExchange (20) - : : +- * Filter (19) - : : +- * ColumnarToRow (18) - : : +- Scan parquet default.store (17) + : : +- * Project (19) + : : +- * Filter (18) + : : +- * ColumnarToRow (17) + : : +- Scan parquet default.date_dim (16) : :- * HashAggregate (46) : : +- Exchange (45) : : +- * HashAggregate (44) : : +- * Project (43) : : +- * BroadcastHashJoin Inner BuildRight (42) - : : :- * Project (37) - : : : +- * BroadcastHashJoin Inner BuildRight (36) + : : :- * Project (40) + : : : +- * BroadcastHashJoin Inner BuildRight (39) : : : :- Union (34) : : : : :- * Project (29) : : : : : +- * Filter (28) @@ -49,18 +49,18 @@ TakeOrderedAndProject (94) : : : : +- * Filter (32) : : : : +- * ColumnarToRow (31) : : : : +- Scan parquet default.catalog_returns (30) - : : : +- ReusedExchange (35) - : : +- BroadcastExchange (41) - : : +- * Filter (40) - : : +- * ColumnarToRow (39) - : : +- Scan parquet default.catalog_page (38) + : : : +- BroadcastExchange (38) + : : : +- * Filter (37) + : : : +- * ColumnarToRow (36) + : : : +- Scan parquet default.catalog_page (35) + : : +- ReusedExchange (41) : +- * HashAggregate (75) : +- Exchange (74) : +- * HashAggregate (73) : +- * Project (72) : +- * BroadcastHashJoin Inner BuildRight (71) - : :- * Project (66) - : : +- * BroadcastHashJoin Inner BuildRight (65) + : :- * Project (69) + : : +- * BroadcastHashJoin Inner BuildRight (68) : : :- Union (63) : : : :- * Project (50) : : : : +- * Filter (49) @@ -78,11 +78,11 @@ TakeOrderedAndProject (94) : : : +- * Filter (58) : : : +- * ColumnarToRow (57) : : : +- Scan parquet default.web_sales (56) - : : +- ReusedExchange (64) - : +- BroadcastExchange (70) - : +- * Filter (69) - : +- * ColumnarToRow (68) - : +- Scan parquet default.web_site (67) + : : +- BroadcastExchange (67) + : : +- * Filter (66) + : : +- * ColumnarToRow (65) + : : +- Scan parquet default.web_site (64) + : +- ReusedExchange (70) :- * HashAggregate (84) : +- Exchange (83) : +- * HashAggregate (82) @@ -132,81 +132,81 @@ Input [4]: [sr_returned_date_sk#11, sr_store_sk#12, sr_return_amt#13, sr_net_los (9) Union -(10) Scan parquet default.date_dim -Output [2]: [d_date_sk#21, d_date#22] +(10) Scan parquet default.store +Output [2]: [s_store_sk#21, s_store_id#22] Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (11) ColumnarToRow [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] +Input [2]: [s_store_sk#21, s_store_id#22] (12) Filter [codegen id : 3] -Input [2]: [d_date_sk#21, d_date#22] -Condition : (((isnotnull(d_date#22) AND (d_date#22 >= 10442)) AND (d_date#22 <= 10456)) AND isnotnull(d_date_sk#21)) +Input [2]: [s_store_sk#21, s_store_id#22] +Condition : isnotnull(s_store_sk#21) -(13) Project [codegen id : 3] -Output [1]: [d_date_sk#21] -Input [2]: [d_date_sk#21, d_date#22] +(13) BroadcastExchange +Input [2]: [s_store_sk#21, s_store_id#22] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#23] -(14) BroadcastExchange -Input [1]: [d_date_sk#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#23] - -(15) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [date_sk#6] -Right keys [1]: [cast(d_date_sk#21 as bigint)] +(14) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [store_sk#5] +Right keys [1]: [cast(s_store_sk#21 as bigint)] Join condition: None -(16) Project [codegen id : 5] -Output [5]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10] -Input [7]: [store_sk#5, date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, d_date_sk#21] +(15) Project [codegen id : 5] +Output [6]: [date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Input [8]: [store_sk#5, date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_sk#21, s_store_id#22] -(17) Scan parquet default.store -Output [2]: [s_store_sk#24, s_store_id#25] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#24, d_date#25] Batched: true -Location [not included in comparison]/{warehouse_dir}/store] -PushedFilters: [IsNotNull(s_store_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] +ReadSchema: struct + +(17) ColumnarToRow [codegen id : 4] +Input [2]: [d_date_sk#24, d_date#25] -(18) ColumnarToRow [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] +(18) Filter [codegen id : 4] +Input [2]: [d_date_sk#24, d_date#25] +Condition : (((isnotnull(d_date#25) AND (d_date#25 >= 10442)) AND (d_date#25 <= 10456)) AND isnotnull(d_date_sk#24)) -(19) Filter [codegen id : 4] -Input [2]: [s_store_sk#24, s_store_id#25] -Condition : isnotnull(s_store_sk#24) +(19) Project [codegen id : 4] +Output [1]: [d_date_sk#24] +Input [2]: [d_date_sk#24, d_date#25] (20) BroadcastExchange -Input [2]: [s_store_sk#24, s_store_id#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [1]: [d_date_sk#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (21) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [store_sk#5] -Right keys [1]: [cast(s_store_sk#24 as bigint)] +Left keys [1]: [date_sk#6] +Right keys [1]: [cast(d_date_sk#24 as bigint)] Join condition: None (22) Project [codegen id : 5] -Output [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Input [7]: [store_sk#5, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_sk#24, s_store_id#25] +Output [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Input [7]: [date_sk#6, sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22, d_date_sk#24] (23) HashAggregate [codegen id : 5] -Input [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#25] -Keys [1]: [s_store_id#25] +Input [5]: [sales_price#7, profit#8, return_amt#9, net_loss#10, s_store_id#22] +Keys [1]: [s_store_id#22] Functions [4]: [partial_sum(UnscaledValue(sales_price#7)), partial_sum(UnscaledValue(return_amt#9)), partial_sum(UnscaledValue(profit#8)), partial_sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum#27, sum#28, sum#29, sum#30] -Results [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] +Results [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] (24) Exchange -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Arguments: hashpartitioning(s_store_id#25, 5), ENSURE_REQUIREMENTS, [id=#35] +Input [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] +Arguments: hashpartitioning(s_store_id#22, 5), ENSURE_REQUIREMENTS, [id=#35] (25) HashAggregate [codegen id : 6] -Input [5]: [s_store_id#25, sum#31, sum#32, sum#33, sum#34] -Keys [1]: [s_store_id#25] +Input [5]: [s_store_id#22, sum#31, sum#32, sum#33, sum#34] +Keys [1]: [s_store_id#22] Functions [4]: [sum(UnscaledValue(sales_price#7)), sum(UnscaledValue(return_amt#9)), sum(UnscaledValue(profit#8)), sum(UnscaledValue(net_loss#10))] Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#7))#36, sum(UnscaledValue(return_amt#9))#37, sum(UnscaledValue(profit#8))#38, sum(UnscaledValue(net_loss#10))#39] -Results [5]: [store channel AS channel#40, concat(store, s_store_id#25) AS id#41, MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS returns#43, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#44] +Results [5]: [store channel AS channel#40, concat(store, s_store_id#22) AS id#41, MakeDecimal(sum(UnscaledValue(sales_price#7))#36,17,2) AS sales#42, MakeDecimal(sum(UnscaledValue(return_amt#9))#37,17,2) AS returns#43, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#8))#38,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#10))#39,17,2) as decimal(18,2)))), DecimalType(18,2), true) AS profit#44] (26) Scan parquet default.catalog_sales Output [4]: [cs_sold_date_sk#45, cs_catalog_page_sk#46, cs_ext_sales_price#47, cs_net_profit#48] @@ -246,44 +246,44 @@ Input [4]: [cr_returned_date_sk#55, cr_catalog_page_sk#56, cr_return_amount#57, (34) Union -(35) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(36) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [date_sk#50] -Right keys [1]: [d_date_sk#21] -Join condition: None - -(37) Project [codegen id : 11] -Output [5]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54] -Input [7]: [page_sk#49, date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, d_date_sk#21] - -(38) Scan parquet default.catalog_page +(35) Scan parquet default.catalog_page Output [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(39) ColumnarToRow [codegen id : 10] +(36) ColumnarToRow [codegen id : 9] Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] -(40) Filter [codegen id : 10] +(37) Filter [codegen id : 9] Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Condition : isnotnull(cp_catalog_page_sk#65) -(41) BroadcastExchange +(38) BroadcastExchange Input [2]: [cp_catalog_page_sk#65, cp_catalog_page_id#66] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#67] -(42) BroadcastHashJoin [codegen id : 11] +(39) BroadcastHashJoin [codegen id : 11] Left keys [1]: [page_sk#49] Right keys [1]: [cp_catalog_page_sk#65] Join condition: None +(40) Project [codegen id : 11] +Output [6]: [date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] +Input [8]: [page_sk#49, date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_sk#65, cp_catalog_page_id#66] + +(41) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#24] + +(42) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [date_sk#50] +Right keys [1]: [d_date_sk#24] +Join condition: None + (43) Project [codegen id : 11] Output [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] -Input [7]: [page_sk#49, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_sk#65, cp_catalog_page_id#66] +Input [7]: [date_sk#50, sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66, d_date_sk#24] (44) HashAggregate [codegen id : 11] Input [5]: [sales_price#51, profit#52, return_amt#53, net_loss#54, cp_catalog_page_id#66] @@ -376,44 +376,44 @@ Input [8]: [wr_returned_date_sk#96, wr_item_sk#97, wr_order_number#98, wr_return (63) Union -(64) ReusedExchange [Reuses operator id: 14] -Output [1]: [d_date_sk#21] - -(65) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [date_sk#91] -Right keys [1]: [cast(d_date_sk#21 as bigint)] -Join condition: None - -(66) Project [codegen id : 21] -Output [5]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95] -Input [7]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, d_date_sk#21] - -(67) Scan parquet default.web_site +(64) Scan parquet default.web_site Output [2]: [web_site_sk#111, web_site_id#112] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(68) ColumnarToRow [codegen id : 20] +(65) ColumnarToRow [codegen id : 19] Input [2]: [web_site_sk#111, web_site_id#112] -(69) Filter [codegen id : 20] +(66) Filter [codegen id : 19] Input [2]: [web_site_sk#111, web_site_id#112] Condition : isnotnull(web_site_sk#111) -(70) BroadcastExchange +(67) BroadcastExchange Input [2]: [web_site_sk#111, web_site_id#112] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#113] -(71) BroadcastHashJoin [codegen id : 21] +(68) BroadcastHashJoin [codegen id : 21] Left keys [1]: [wsr_web_site_sk#90] Right keys [1]: [web_site_sk#111] Join condition: None +(69) Project [codegen id : 21] +Output [6]: [date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] +Input [8]: [wsr_web_site_sk#90, date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#111, web_site_id#112] + +(70) ReusedExchange [Reuses operator id: 20] +Output [1]: [d_date_sk#24] + +(71) BroadcastHashJoin [codegen id : 21] +Left keys [1]: [date_sk#91] +Right keys [1]: [cast(d_date_sk#24 as bigint)] +Join condition: None + (72) Project [codegen id : 21] Output [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] -Input [7]: [wsr_web_site_sk#90, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_sk#111, web_site_id#112] +Input [7]: [date_sk#91, sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112, d_date_sk#24] (73) HashAggregate [codegen id : 21] Input [5]: [sales_price#92, profit#93, return_amt#94, net_loss#95, web_site_id#112] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt index 233af6d8cc..8d1794b903 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/simplified.txt @@ -22,9 +22,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (5) HashAggregate [s_store_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,s_store_id] - BroadcastHashJoin [store_sk,s_store_sk] - Project [store_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,s_store_id] + BroadcastHashJoin [store_sk,s_store_sk] InputAdapter Union WholeStageCodegen (1) @@ -42,18 +42,18 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter BroadcastExchange #4 WholeStageCodegen (3) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk,s_store_id] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk,s_store_id] + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] WholeStageCodegen (12) HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter @@ -61,9 +61,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (11) HashAggregate [cp_catalog_page_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - BroadcastHashJoin [page_sk,cp_catalog_page_sk] - Project [page_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + BroadcastHashJoin [page_sk,cp_catalog_page_sk] InputAdapter Union WholeStageCodegen (7) @@ -79,14 +79,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.catalog_returns [cr_returned_date_sk,cr_catalog_page_sk,cr_return_amount,cr_net_loss] InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #7 + WholeStageCodegen (9) + Filter [cp_catalog_page_sk] + ColumnarToRow + InputAdapter + Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (10) - Filter [cp_catalog_page_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [d_date_sk] #5 WholeStageCodegen (22) HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] InputAdapter @@ -94,9 +94,9 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] WholeStageCodegen (21) HashAggregate [web_site_id,sales_price,return_amt,profit,net_loss] [sum,sum,sum,sum,sum,sum,sum,sum] Project [sales_price,profit,return_amt,net_loss,web_site_id] - BroadcastHashJoin [wsr_web_site_sk,web_site_sk] - Project [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - BroadcastHashJoin [date_sk,d_date_sk] + BroadcastHashJoin [date_sk,d_date_sk] + Project [date_sk,sales_price,profit,return_amt,net_loss,web_site_id] + BroadcastHashJoin [wsr_web_site_sk,web_site_sk] InputAdapter Union WholeStageCodegen (13) @@ -129,14 +129,14 @@ TakeOrderedAndProject [channel,id,sales,returns,profit] InputAdapter Scan parquet default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number] InputAdapter - ReusedExchange [d_date_sk] #4 + BroadcastExchange #11 + WholeStageCodegen (19) + Filter [web_site_sk] + ColumnarToRow + InputAdapter + Scan parquet default.web_site [web_site_sk,web_site_id] InputAdapter - BroadcastExchange #11 - WholeStageCodegen (20) - Filter [web_site_sk] - ColumnarToRow - InputAdapter - Scan parquet default.web_site [web_site_sk,web_site_id] + ReusedExchange [d_date_sk] #5 WholeStageCodegen (49) HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt index a7f328537b..04ff822b1c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/explain.txt @@ -11,60 +11,60 @@ TakeOrderedAndProject (79) : +- * BroadcastHashJoin LeftOuter BuildRight (65) : :- * Project (60) : : +- * SortMergeJoin Inner (59) - : : :- * Sort (47) - : : : +- Exchange (46) - : : : +- * Project (45) - : : : +- * BroadcastHashJoin Inner BuildRight (44) - : : : :- * Project (32) - : : : : +- * SortMergeJoin Inner (31) - : : : : :- * Sort (25) - : : : : : +- Exchange (24) - : : : : : +- * Project (23) - : : : : : +- * BroadcastHashJoin Inner BuildRight (22) - : : : : : :- * Project (17) - : : : : : : +- * BroadcastHashJoin Inner BuildRight (16) - : : : : : : :- * Project (10) - : : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) - : : : : : : : :- * Filter (3) - : : : : : : : : +- * ColumnarToRow (2) - : : : : : : : : +- Scan parquet default.catalog_sales (1) - : : : : : : : +- BroadcastExchange (8) - : : : : : : : +- * Project (7) - : : : : : : : +- * Filter (6) - : : : : : : : +- * ColumnarToRow (5) - : : : : : : : +- Scan parquet default.household_demographics (4) - : : : : : : +- BroadcastExchange (15) - : : : : : : +- * Project (14) - : : : : : : +- * Filter (13) - : : : : : : +- * ColumnarToRow (12) - : : : : : : +- Scan parquet default.customer_demographics (11) - : : : : : +- BroadcastExchange (21) - : : : : : +- * Filter (20) - : : : : : +- * ColumnarToRow (19) - : : : : : +- Scan parquet default.date_dim (18) - : : : : +- * Sort (30) - : : : : +- Exchange (29) - : : : : +- * Filter (28) - : : : : +- * ColumnarToRow (27) - : : : : +- Scan parquet default.item (26) - : : : +- BroadcastExchange (43) - : : : +- * Project (42) - : : : +- * BroadcastHashJoin Inner BuildLeft (41) - : : : :- BroadcastExchange (37) - : : : : +- * Project (36) - : : : : +- * Filter (35) - : : : : +- * ColumnarToRow (34) - : : : : +- Scan parquet default.date_dim (33) - : : : +- * Filter (40) - : : : +- * ColumnarToRow (39) - : : : +- Scan parquet default.date_dim (38) + : : :- * Sort (34) + : : : +- Exchange (33) + : : : +- * Project (32) + : : : +- * SortMergeJoin Inner (31) + : : : :- * Sort (25) + : : : : +- Exchange (24) + : : : : +- * Project (23) + : : : : +- * BroadcastHashJoin Inner BuildRight (22) + : : : : :- * Project (17) + : : : : : +- * BroadcastHashJoin Inner BuildRight (16) + : : : : : :- * Project (10) + : : : : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : : : : :- * Filter (3) + : : : : : : : +- * ColumnarToRow (2) + : : : : : : : +- Scan parquet default.catalog_sales (1) + : : : : : : +- BroadcastExchange (8) + : : : : : : +- * Project (7) + : : : : : : +- * Filter (6) + : : : : : : +- * ColumnarToRow (5) + : : : : : : +- Scan parquet default.household_demographics (4) + : : : : : +- BroadcastExchange (15) + : : : : : +- * Project (14) + : : : : : +- * Filter (13) + : : : : : +- * ColumnarToRow (12) + : : : : : +- Scan parquet default.customer_demographics (11) + : : : : +- BroadcastExchange (21) + : : : : +- * Filter (20) + : : : : +- * ColumnarToRow (19) + : : : : +- Scan parquet default.date_dim (18) + : : : +- * Sort (30) + : : : +- Exchange (29) + : : : +- * Filter (28) + : : : +- * ColumnarToRow (27) + : : : +- Scan parquet default.item (26) : : +- * Sort (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * Filter (50) - : : : +- * ColumnarToRow (49) - : : : +- Scan parquet default.inventory (48) + : : :- * Project (50) + : : : +- * BroadcastHashJoin Inner BuildLeft (49) + : : : :- BroadcastExchange (45) + : : : : +- * Project (44) + : : : : +- * BroadcastHashJoin Inner BuildLeft (43) + : : : : :- BroadcastExchange (39) + : : : : : +- * Project (38) + : : : : : +- * Filter (37) + : : : : : +- * ColumnarToRow (36) + : : : : : +- Scan parquet default.date_dim (35) + : : : : +- * Filter (42) + : : : : +- * ColumnarToRow (41) + : : : : +- Scan parquet default.date_dim (40) + : : : +- * Filter (48) + : : : +- * ColumnarToRow (47) + : : : +- Scan parquet default.inventory (46) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -185,7 +185,7 @@ Input [8]: [cs_sold_date_sk#1, cs_ship_date_sk#2, cs_item_sk#5, cs_promo_sk#6, c (24) Exchange Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] -Arguments: hashpartitioning(cs_item_sk#5, 5), true, [id=#18] +Arguments: hashpartitioning(cs_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#18] (25) Sort [codegen id : 5] Input [6]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16] @@ -207,101 +207,101 @@ Condition : isnotnull(i_item_sk#19) (29) Exchange Input [2]: [i_item_sk#19, i_item_desc#20] -Arguments: hashpartitioning(i_item_sk#19, 5), true, [id=#21] +Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] (30) Sort [codegen id : 7] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 10] +(31) SortMergeJoin [codegen id : 8] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 10] +(32) Project [codegen id : 8] Output [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] Input [8]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_sk#19, i_item_desc#20] -(33) Scan parquet default.date_dim -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(33) Exchange +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: hashpartitioning(cs_item_sk#5, cs_sold_date_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] + +(34) Sort [codegen id : 9] +Input [7]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20] +Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_sold_date_sk#1 ASC NULLS FIRST], false, 0 + +(35) Scan parquet default.date_dim +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk), IsNotNull(d_week_seq), IsNotNull(d_date)] ReadSchema: struct -(34) ColumnarToRow [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(36) ColumnarToRow [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(35) Filter [codegen id : 8] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -Condition : ((((isnotnull(d_year#25) AND (d_year#25 = 2001)) AND isnotnull(d_date_sk#22)) AND isnotnull(d_week_seq#24)) AND isnotnull(d_date#23)) +(37) Filter [codegen id : 10] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +Condition : ((((isnotnull(d_year#26) AND (d_year#26 = 2001)) AND isnotnull(d_date_sk#23)) AND isnotnull(d_week_seq#25)) AND isnotnull(d_date#24)) -(36) Project [codegen id : 8] -Output [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] +(38) Project [codegen id : 10] +Output [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] -(37) BroadcastExchange -Input [3]: [d_date_sk#22, d_date#23, d_week_seq#24] -Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#26] +(39) BroadcastExchange +Input [3]: [d_date_sk#23, d_date#24, d_week_seq#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[2, int, true] as bigint)),false), [id=#27] -(38) Scan parquet default.date_dim -Output [2]: [d_date_sk#27, d_week_seq#28] +(40) Scan parquet default.date_dim +Output [2]: [d_date_sk#28, d_week_seq#29] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(39) ColumnarToRow -Input [2]: [d_date_sk#27, d_week_seq#28] +(41) ColumnarToRow +Input [2]: [d_date_sk#28, d_week_seq#29] -(40) Filter -Input [2]: [d_date_sk#27, d_week_seq#28] -Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) +(42) Filter +Input [2]: [d_date_sk#28, d_week_seq#29] +Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) -(41) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [d_week_seq#24] -Right keys [1]: [d_week_seq#28] +(43) BroadcastHashJoin [codegen id : 11] +Left keys [1]: [d_week_seq#25] +Right keys [1]: [d_week_seq#29] Join condition: None -(42) Project [codegen id : 9] -Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Input [5]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27, d_week_seq#28] - -(43) BroadcastExchange -Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#29] - -(44) BroadcastHashJoin [codegen id : 10] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] -Join condition: (d_date#16 > d_date#23 + 5 days) - -(45) Project [codegen id : 10] -Output [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Input [11]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#22, d_date#23, d_week_seq#24, d_date_sk#27] +(44) Project [codegen id : 11] +Output [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Input [5]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, d_week_seq#29] -(46) Exchange -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: hashpartitioning(cs_item_sk#5, d_date_sk#27, 5), true, [id=#30] +(45) BroadcastExchange +Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28] +Arguments: HashedRelationBroadcastMode(List(cast(input[3, int, true] as bigint)),false), [id=#30] -(47) Sort [codegen id : 11] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27] -Arguments: [cs_item_sk#5 ASC NULLS FIRST, d_date_sk#27 ASC NULLS FIRST], false, 0 - -(48) Scan parquet default.inventory +(46) Scan parquet default.inventory Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Batched: true Location [not included in comparison]/{warehouse_dir}/inventory] PushedFilters: [IsNotNull(inv_quantity_on_hand), IsNotNull(inv_item_sk), IsNotNull(inv_warehouse_sk), IsNotNull(inv_date_sk)] ReadSchema: struct -(49) ColumnarToRow [codegen id : 13] +(47) ColumnarToRow Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(50) Filter [codegen id : 13] +(48) Filter Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] Condition : (((isnotnull(inv_quantity_on_hand#34) AND isnotnull(inv_item_sk#32)) AND isnotnull(inv_warehouse_sk#33)) AND isnotnull(inv_date_sk#31)) +(49) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [d_date_sk#28] +Right keys [1]: [inv_date_sk#31] +Join condition: None + +(50) Project [codegen id : 13] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, d_date_sk#28, inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] + (51) Scan parquet default.warehouse Output [2]: [w_warehouse_sk#35, w_warehouse_name#36] Batched: true @@ -326,25 +326,25 @@ Right keys [1]: [w_warehouse_sk#35] Join condition: None (56) Project [codegen id : 13] -Output [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Input [6]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] +Output [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Input [8]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34, w_warehouse_sk#35, w_warehouse_name#36] (57) Exchange -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: hashpartitioning(inv_item_sk#32, inv_date_sk#31, 5), true, [id=#38] +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: hashpartitioning(inv_item_sk#32, d_date_sk#23, 5), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] -Arguments: [inv_item_sk#32 ASC NULLS FIRST, inv_date_sk#31 ASC NULLS FIRST], false, 0 +Input [6]: [d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Arguments: [inv_item_sk#32 ASC NULLS FIRST, d_date_sk#23 ASC NULLS FIRST], false, 0 (59) SortMergeJoin [codegen id : 16] -Left keys [2]: [cs_item_sk#5, d_date_sk#27] -Right keys [2]: [inv_item_sk#32, inv_date_sk#31] -Join condition: (inv_quantity_on_hand#34 < cs_quantity#8) +Left keys [2]: [cs_item_sk#5, cs_sold_date_sk#1] +Right keys [2]: [inv_item_sk#32, d_date_sk#23] +Join condition: ((inv_quantity_on_hand#34 < cs_quantity#8) AND (d_date#16 > d_date#24 + 5 days)) (60) Project [codegen id : 16] -Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [11]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, i_item_desc#20, d_week_seq#24, d_date_sk#27, inv_date_sk#31, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] +Output [6]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [13]: [cs_sold_date_sk#1, cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, cs_quantity#8, d_date#16, i_item_desc#20, d_date_sk#23, d_date#24, d_week_seq#25, inv_item_sk#32, inv_quantity_on_hand#34, w_warehouse_name#36] (61) Scan parquet default.promotion Output [1]: [p_promo_sk#39] @@ -370,15 +370,15 @@ Right keys [1]: [p_promo_sk#39] Join condition: None (66) Project [codegen id : 16] -Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, p_promo_sk#39] +Output [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_promo_sk#6, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, p_promo_sk#39] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), true, [id=#41] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Arguments: hashpartitioning(cs_item_sk#5, cs_order_number#7, 5), ENSURE_REQUIREMENTS, [id=#41] (68) Sort [codegen id : 17] -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -397,7 +397,7 @@ Condition : (isnotnull(cr_item_sk#42) AND isnotnull(cr_order_number#43)) (72) Exchange Input [2]: [cr_item_sk#42, cr_order_number#43] -Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), true, [id=#44] +Arguments: hashpartitioning(cr_item_sk#42, cr_order_number#43, 5), ENSURE_REQUIREMENTS, [id=#44] (73) Sort [codegen id : 19] Input [2]: [cr_item_sk#42, cr_order_number#43] @@ -409,28 +409,28 @@ Right keys [2]: [cr_item_sk#42, cr_order_number#43] Join condition: None (75) Project [codegen id : 20] -Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24, cr_item_sk#42, cr_order_number#43] +Output [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Input [7]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25, cr_item_sk#42, cr_order_number#43] (76) HashAggregate [codegen id : 20] -Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#24] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [3]: [w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] (77) Exchange -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#24, 5), true, [id=#47] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Arguments: hashpartitioning(i_item_desc#20, w_warehouse_name#36, d_week_seq#25, 5), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] -Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24] +Input [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Keys [3]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] +Results [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count(1)#48 AS no_promo#49, count(1)#48 AS promo#50, count(1)#48 AS total_cnt#51] (79) TakeOrderedAndProject -Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] -Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#24 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, no_promo#49, promo#50, total_cnt#51] +Input [6]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] +Arguments: 100, [total_cnt#51 DESC NULLS LAST, i_item_desc#20 ASC NULLS FIRST, w_warehouse_name#36 ASC NULLS FIRST, d_week_seq#25 ASC NULLS FIRST], [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, no_promo#49, promo#50, total_cnt#51] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt index 918508787c..b88505ad7b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q72.sf100/simplified.txt @@ -16,95 +16,95 @@ TakeOrderedAndProject [total_cnt,i_item_desc,w_warehouse_name,d_week_seq,no_prom Project [cs_item_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] BroadcastHashJoin [cs_promo_sk,p_promo_sk] Project [cs_item_sk,cs_promo_sk,cs_order_number,w_warehouse_name,i_item_desc,d_week_seq] - SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] + SortMergeJoin [cs_item_sk,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] InputAdapter - WholeStageCodegen (11) - Sort [cs_item_sk,d_date_sk] + WholeStageCodegen (9) + Sort [cs_item_sk,cs_sold_date_sk] InputAdapter - Exchange [cs_item_sk,d_date_sk] #3 - WholeStageCodegen (10) - Project [cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,i_item_desc,d_week_seq,d_date_sk] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk,d_date,d_date] - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (5) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #4 - WholeStageCodegen (4) - Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] - BroadcastHashJoin [cs_ship_date_sk,d_date_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] - Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] - BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] - Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + Exchange [cs_item_sk,cs_sold_date_sk] #3 + WholeStageCodegen (8) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date,i_item_desc] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (5) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #4 + WholeStageCodegen (4) + Project [cs_sold_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity,d_date] + BroadcastHashJoin [cs_ship_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_cdemo_sk,cd_demo_sk] + Project [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] + BroadcastHashJoin [cs_bill_hdemo_sk,hd_demo_sk] + Filter [cs_quantity,cs_item_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_sold_date_sk,cs_ship_date_sk] + ColumnarToRow InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] - ColumnarToRow - InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.catalog_sales [cs_sold_date_sk,cs_ship_date_sk,cs_bill_cdemo_sk,cs_bill_hdemo_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_quantity] InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [cd_demo_sk] - Filter [cd_marital_status,cd_demo_sk] + BroadcastExchange #5 + WholeStageCodegen (1) + Project [hd_demo_sk] + Filter [hd_buy_potential,hd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] InputAdapter - BroadcastExchange #7 - WholeStageCodegen (3) - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] - InputAdapter - WholeStageCodegen (7) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #8 - WholeStageCodegen (6) - Filter [i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_desc] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] + ColumnarToRow + InputAdapter + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (3) + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [d_date_sk,d_date,d_week_seq,d_date_sk] - BroadcastHashJoin [d_week_seq,d_week_seq] - InputAdapter - BroadcastExchange #10 - WholeStageCodegen (8) - Project [d_date_sk,d_date,d_week_seq] - Filter [d_year,d_date_sk,d_week_seq,d_date] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] - Filter [d_week_seq,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_week_seq] + WholeStageCodegen (7) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #8 + WholeStageCodegen (6) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_desc] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,inv_date_sk] + Sort [inv_item_sk,d_date_sk] InputAdapter - Exchange [inv_item_sk,inv_date_sk] #11 + Exchange [inv_item_sk,d_date_sk] #9 WholeStageCodegen (13) - Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] - ColumnarToRow + Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastHashJoin [d_date_sk,inv_date_sk] InputAdapter - Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] + BroadcastExchange #10 + WholeStageCodegen (11) + Project [d_date_sk,d_date,d_week_seq,d_date_sk] + BroadcastHashJoin [d_week_seq,d_week_seq] + InputAdapter + BroadcastExchange #11 + WholeStageCodegen (10) + Project [d_date_sk,d_date,d_week_seq] + Filter [d_year,d_date_sk,d_week_seq,d_date] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date,d_week_seq,d_year] + Filter [d_week_seq,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_week_seq] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.inventory [inv_date_sk,inv_item_sk,inv_warehouse_sk,inv_quantity_on_hand] InputAdapter BroadcastExchange #12 WholeStageCodegen (12) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala index 62d0f51e5f..fb35d6cf8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CharVarcharTestSuite.scala @@ -451,6 +451,29 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { Seq(Row("char(5)"), Row("varchar(3)"))) } } + + test("SPARK-33992: char/varchar resolution in correlated sub query") { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") + sql(s"CREATE TABLE t2(v VARCHAR(3), c CHAR(5)) USING $format") + sql("INSERT INTO t1 VALUES ('c', 'b')") + sql("INSERT INTO t2 VALUES ('a', 'b')") + + checkAnswer(sql( + """ + |SELECT v FROM t1 + |WHERE 'a' IN (SELECT v FROM t2 WHERE t1.c = t2.c )""".stripMargin), + Row("c")) + } + } + + test("SPARK-34003: fix char/varchar fails w/ both group by and order by ") { + withTable("t") { + sql(s"CREATE TABLE t(v VARCHAR(3), i INT) USING $format") + sql("INSERT INTO t VALUES ('c', 1)") + checkAnswer(sql("SELECT v, sum(i) FROM t GROUP BY v ORDER BY v"), Row("c", 1)) + } + } } // Some basic char/varchar tests which doesn't rely on table implementation. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 1bdfdb5ab9..2e336b264c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -1106,20 +1106,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } test("SPARK-32330: Preserve shuffled hash join build side partitioning") { - withSQLConf( - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", - SQLConf.SHUFFLE_PARTITIONS.key -> "2", - SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { - val df1 = spark.range(10).select($"id".as("k1")) - val df2 = spark.range(30).select($"id".as("k2")) - Seq("inner", "cross").foreach(joinType => { - val plan = df1.join(df2, $"k1" === $"k2", joinType).groupBy($"k1").count() - .queryExecution.executedPlan - assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) - // No extra shuffle before aggregate - assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 2) - }) - } + val df1 = spark.range(10).select($"id".as("k1")) + val df2 = spark.range(30).select($"id".as("k2")) + Seq("inner", "cross").foreach(joinType => { + val plan = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) + .groupBy($"k1").count() + .queryExecution.executedPlan + assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra shuffle before aggregate + assert(collect(plan) { case _: ShuffleExchangeExec => true }.size === 2) + }) } test("SPARK-32383: Preserve hash join (BHJ and SHJ) stream side ordering") { @@ -1129,40 +1125,30 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan val df4 = spark.range(100).select($"id".as("k4")) // Test broadcast hash join - withSQLConf( - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50") { - Seq("inner", "left_outer").foreach(joinType => { - val plan = df1.join(df2, $"k1" === $"k2", joinType) - .join(df3, $"k1" === $"k3", joinType) - .join(df4, $"k1" === $"k4", joinType) - .queryExecution - .executedPlan - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) - assert(collect(plan) { case _: BroadcastHashJoinExec => true }.size === 1) - // No extra sort before last sort merge join - assert(collect(plan) { case _: SortExec => true }.size === 3) - }) - } + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", joinType) + .join(df3.hint("BROADCAST"), $"k1" === $"k3", joinType) + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) + assert(collect(plan) { case _: BroadcastHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(collect(plan) { case _: SortExec => true }.size === 3) + }) // Test shuffled hash join - withSQLConf( - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", - SQLConf.SHUFFLE_PARTITIONS.key -> "2", - SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { - val df3 = spark.range(10).select($"id".as("k3")) - - Seq("inner", "left_outer").foreach(joinType => { - val plan = df1.join(df2, $"k1" === $"k2", joinType) - .join(df3, $"k1" === $"k3", joinType) - .join(df4, $"k1" === $"k4", joinType) - .queryExecution - .executedPlan - assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) - assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) - // No extra sort before last sort merge join - assert(collect(plan) { case _: SortExec => true }.size === 3) - }) - } + Seq("inner", "left_outer").foreach(joinType => { + val plan = df1.join(df2.hint("SHUFFLE_MERGE"), $"k1" === $"k2", joinType) + .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3", joinType) + .join(df4.hint("SHUFFLE_MERGE"), $"k1" === $"k4", joinType) + .queryExecution + .executedPlan + assert(collect(plan) { case _: SortMergeJoinExec => true }.size === 2) + assert(collect(plan) { case _: ShuffledHashJoinExec => true }.size === 1) + // No extra sort before last sort merge join + assert(collect(plan) { case _: SortExec => true }.size === 3) + }) } test("SPARK-32290: SingleColumn Null Aware Anti Join Optimize") { @@ -1250,24 +1236,16 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan $"k1" === $"k4" && $"k2" === $"k5" && $"k3" === $"k6") ) inputDFs.foreach { case (df1, df2, joinExprs) => - withSQLConf( - // Set broadcast join threshold and number of shuffle partitions, - // as shuffled hash join depends on these two configs. - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", - SQLConf.SHUFFLE_PARTITIONS.key -> "2") { - val smjDF = df1.join(df2, joinExprs, "full") - assert(collect(smjDF.queryExecution.executedPlan) { - case _: SortMergeJoinExec => true }.size === 1) - val smjResult = smjDF.collect() - - withSQLConf(SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { - val shjDF = df1.join(df2, joinExprs, "full") - assert(collect(shjDF.queryExecution.executedPlan) { - case _: ShuffledHashJoinExec => true }.size === 1) - // Same result between shuffled hash join and sort merge join - checkAnswer(shjDF, smjResult) - } - } + val smjDF = df1.join(df2.hint("SHUFFLE_MERGE"), joinExprs, "full") + assert(collect(smjDF.queryExecution.executedPlan) { + case _: SortMergeJoinExec => true }.size === 1) + val smjResult = smjDF.collect() + + val shjDF = df1.join(df2.hint("SHUFFLE_HASH"), joinExprs, "full") + assert(collect(shjDF.queryExecution.executedPlan) { + case _: ShuffledHashJoinExec => true }.size === 1) + // Same result between shuffled hash join and sort merge join + checkAnswer(shjDF, smjResult) } } @@ -1284,10 +1262,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan ) inputDFs.foreach { case (df1, df2, joinType) => // Test broadcast hash join - withSQLConf( - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "200", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { - val bhjCodegenDF = df1.join(df2, $"k1" === $"k2", joinType) + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val bhjCodegenDF = df1.join(df2.hint("BROADCAST"), $"k1" === $"k2", joinType) assert(bhjCodegenDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : BroadcastHashJoinExec) => true case WholeStageCodegenExec(ProjectExec(_, _ : BroadcastHashJoinExec)) => true @@ -1303,13 +1279,8 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan } // Test shuffled hash join - withSQLConf(SQLConf.PREFER_SORTMERGEJOIN.key -> "false", - // Set broadcast join threshold and number of shuffle partitions, - // as shuffled hash join depends on these two configs. - SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "50", - SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", - SQLConf.SHUFFLE_PARTITIONS.key -> "2") { - val shjCodegenDF = df1.join(df2, $"k1" === $"k2", joinType) + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val shjCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) assert(shjCodegenDF.queryExecution.executedPlan.collect { case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true case WholeStageCodegenExec(ProjectExec(_, _ : ShuffledHashJoinExec)) => true @@ -1317,7 +1288,7 @@ class JoinSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlan checkAnswer(shjCodegenDF, Seq.empty) withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { - val shjNonCodegenDF = df1.join(df2, $"k1" === $"k2", joinType) + val shjNonCodegenDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2", joinType) assert(shjNonCodegenDF.queryExecution.executedPlan.collect { case _: ShuffledHashJoinExec => true }.size === 1) checkAnswer(shjNonCodegenDF, Seq.empty) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 44f3c3449d..d46db8f995 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -3768,6 +3768,30 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } + test("Fold RepartitionExpression num partition should check if partition expression is empty") { + withSQLConf((SQLConf.SHUFFLE_PARTITIONS.key, "5")) { + val df = spark.range(1).hint("REPARTITION_BY_RANGE") + val plan = df.queryExecution.optimizedPlan + val res = plan.collect { + case r: RepartitionByExpression if r.numPartitions == 5 => true + } + assert(res.nonEmpty) + } + } + + test("SPARK-34030: Fold RepartitionExpression num partition should at Optimizer") { + withSQLConf((SQLConf.SHUFFLE_PARTITIONS.key, "2")) { + Seq(1, "1, 2", null, "version()").foreach { expr => + val plan = sql(s"select * from values (1), (2), (3) t(a) distribute by $expr") + .queryExecution.analyzed + val res = plan.collect { + case r: RepartitionByExpression if r.numPartitions == 2 => true + } + assert(res.nonEmpty) + } + } + } + test("SPARK-33593: Vector reader got incorrect data with binary partition value") { Seq("false", "true").foreach(value => { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> value) { @@ -3843,6 +3867,15 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark assert(unions.size == 1) } + + test("SPARK-33591: null as a partition value") { + val t = "part_table" + withTable(t) { + sql(s"CREATE TABLE $t (col1 INT, p1 STRING) USING PARQUET PARTITIONED BY (p1)") + sql(s"INSERT INTO TABLE $t PARTITION (p1 = null) SELECT 0") + checkAnswer(sql(s"SELECT * FROM $t"), Row(0, null)) + } + } } case class Foo(bar: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 92d306c0e3..5ce5d36c5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -223,29 +223,6 @@ abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { } protected def checkCatalogTables(expected: CatalogTable, actual: CatalogTable): Unit = { - def normalize(table: CatalogTable): CatalogTable = { - val nondeterministicProps = Set( - "CreateTime", - "transient_lastDdlTime", - "grantTime", - "lastUpdateTime", - "last_modified_by", - "last_modified_time", - "Owner:", - // The following are hive specific schema parameters which we do not need to match exactly. - "totalNumberFiles", - "maxFileSize", - "minFileSize" - ) - - table.copy( - createTime = 0L, - lastAccessTime = 0L, - properties = table.properties.filterKeys(!nondeterministicProps.contains(_)).toMap, - stats = None, - ignoredProperties = Map.empty - ) - } - assert(normalize(actual) == normalize(expected)) + assert(CatalogTable.normalize(actual) == CatalogTable.normalize(expected)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 47829b68cc..0a6bd795cd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.internal.SQLConf.{PARTITION_OVERWRITE_MODE, PartitionOverwriteMode, V2_SESSION_CATALOG_IMPLEMENTATION} import org.apache.spark.sql.internal.connector.SimpleTableProvider import org.apache.spark.sql.sources.SimpleScanSource -import org.apache.spark.sql.types.{BooleanType, LongType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{LongType, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String @@ -752,6 +752,23 @@ class DataSourceV2SQLSuite assert(t2.v1Table.provider == Some(conf.defaultDataSourceName)) } + test("SPARK-34039: ReplaceTable (atomic or non-atomic) should invalidate cache") { + Seq("testcat.ns.t", "testcat_atomic.ns.t").foreach { t => + val view = "view" + withTable(t) { + withTempView(view) { + sql(s"CREATE TABLE $t USING foo AS SELECT id, data FROM source") + sql(s"CACHE TABLE $view AS SELECT id FROM $t") + checkAnswer(sql(s"SELECT * FROM $t"), spark.table("source")) + checkAnswer(sql(s"SELECT * FROM $view"), spark.table("source").select("id")) + + sql(s"REPLACE TABLE $t (a bigint) USING foo") + assert(spark.sharedState.cacheManager.lookupCachedData(spark.table(view)).isEmpty) + } + } + } + } + test("SPARK-33492: ReplaceTableAsSelect (atomic or non-atomic) should invalidate cache") { Seq("testcat.ns.t", "testcat_atomic.ns.t").foreach { t => val view = "view" @@ -991,26 +1008,6 @@ class DataSourceV2SQLSuite " only SessionCatalog supports this command.")) } - private def runShowTablesSql( - sqlText: String, - expected: Seq[Row], - expectV2Catalog: Boolean = true): Unit = { - val schema = if (expectV2Catalog) { - new StructType() - .add("namespace", StringType, nullable = false) - .add("tableName", StringType, nullable = false) - } else { - new StructType() - .add("database", StringType, nullable = false) - .add("tableName", StringType, nullable = false) - .add("isTemporary", BooleanType, nullable = false) - } - - val df = spark.sql(sqlText) - assert(df.schema === schema) - assert(expected === df.collect()) - } - test("CreateNameSpace: basic tests") { // Session catalog is used. withNamespace("ns") { @@ -1305,95 +1302,6 @@ class DataSourceV2SQLSuite } } - test("ShowNamespaces: show root namespaces with default v2 catalog") { - spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat") - - testShowNamespaces("SHOW NAMESPACES", Seq()) - - spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") - - testShowNamespaces("SHOW NAMESPACES", Seq("ns1", "ns2")) - testShowNamespaces("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) - } - - test("ShowNamespaces: show namespaces with v2 catalog") { - spark.sql("CREATE TABLE testcat.ns1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_1.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns1.ns1_2.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.table (id bigint) USING foo") - spark.sql("CREATE TABLE testcat.ns2.ns2_1.table (id bigint) USING foo") - - // Look up only with catalog name, which should list root namespaces. - testShowNamespaces("SHOW NAMESPACES IN testcat", Seq("ns1", "ns2")) - - // Look up sub-namespaces. - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1", Seq("ns1.ns1_1", "ns1.ns1_2")) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1 LIKE '*2*'", Seq("ns1.ns1_2")) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns2", Seq("ns2.ns2_1")) - - // Try to look up namespaces that do not exist. - testShowNamespaces("SHOW NAMESPACES IN testcat.ns3", Seq()) - testShowNamespaces("SHOW NAMESPACES IN testcat.ns1.ns3", Seq()) - } - - test("ShowNamespaces: default v2 catalog is not set") { - spark.sql("CREATE TABLE testcat.ns.table (id bigint) USING foo") - - // The current catalog is resolved to a v2 session catalog. - testShowNamespaces("SHOW NAMESPACES", Seq("default")) - } - - test("ShowNamespaces: default v2 catalog doesn't support namespace") { - spark.conf.set( - "spark.sql.catalog.testcat_no_namespace", - classOf[BasicInMemoryTableCatalog].getName) - spark.conf.set(SQLConf.DEFAULT_CATALOG.key, "testcat_no_namespace") - - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES") - } - - assert(exception.getMessage.contains("does not support namespaces")) - } - - test("ShowNamespaces: v2 catalog doesn't support namespace") { - spark.conf.set( - "spark.sql.catalog.testcat_no_namespace", - classOf[BasicInMemoryTableCatalog].getName) - - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES in testcat_no_namespace") - } - - assert(exception.getMessage.contains("does not support namespaces")) - } - - test("ShowNamespaces: session catalog is used and namespace doesn't exist") { - val exception = intercept[AnalysisException] { - sql("SHOW NAMESPACES in dummy") - } - - assert(exception.getMessage.contains("Namespace 'dummy' not found")) - } - - test("ShowNamespaces: change catalog and namespace with USE statements") { - sql("CREATE TABLE testcat.ns1.ns2.table (id bigint) USING foo") - - // Initially, the current catalog is a v2 session catalog. - testShowNamespaces("SHOW NAMESPACES", Seq("default")) - - // Update the current catalog to 'testcat'. - sql("USE testcat") - testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) - - // Update the current namespace to 'ns1'. - sql("USE ns1") - // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. - testShowNamespaces("SHOW NAMESPACES", Seq("ns1")) - } - private def testShowNamespaces( sqlText: String, expected: Seq[String]): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala index a25e4b8f8e..cef870b249 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BaseScriptTransformationSuite.scala @@ -521,7 +521,10 @@ abstract class BaseScriptTransformationSuite extends SparkPlanTest with SQLTestU 'd.cast("string"), 'e.cast("string")).collect()) }.getMessage - assert(e1.contains("Permission denied")) + // Check with status exit code since in GA test, it may lose detail failed root cause. + // Different root cause's exitcode is not same. + // In this test, root cause is `Permission denied` + assert(e1.contains("Subprocess exited with status 126")) // test `/path/to/script.py' with script executable scriptFilePath.setExecutable(true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index 8aa003a3df..f630cd8322 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -210,23 +210,8 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSparkSession test("SPARK-23376: Create UnsafeKVExternalSorter with BytesToByteMap having duplicated keys") { val memoryManager = new TestMemoryManager(new SparkConf()) val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) - val map = new BytesToBytesMap(taskMemoryManager, 64, taskMemoryManager.pageSizeBytes()) - - // Key/value are a unsafe rows with a single int column + val map = createBytesToBytesMapWithDuplicateKeys(taskMemoryManager) val schema = new StructType().add("i", IntegerType) - val key = new UnsafeRow(1) - key.pointTo(new Array[Byte](32), 32) - key.setInt(0, 1) - val value = new UnsafeRow(1) - value.pointTo(new Array[Byte](32), 32) - value.setInt(0, 2) - - for (_ <- 1 to 65) { - val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes) - loc.append( - key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, - value.getBaseObject, value.getBaseOffset, value.getSizeInBytes) - } // Make sure we can successfully create a UnsafeKVExternalSorter with a `BytesToBytesMap` // which has duplicated keys and the number of entries exceeds its capacity. @@ -245,4 +230,82 @@ class UnsafeKVExternalSorterSuite extends SparkFunSuite with SharedSparkSession TaskContext.unset() } } + + test("SPARK-31952: create UnsafeKVExternalSorter with existing map should count spilled memory " + + "size correctly") { + val memoryManager = new TestMemoryManager(new SparkConf()) + val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + val map = createBytesToBytesMapWithDuplicateKeys(taskMemoryManager) + val schema = new StructType().add("i", IntegerType) + + try { + val context = new TaskContextImpl(0, 0, 0, 0, 0, taskMemoryManager, new Properties(), null) + TaskContext.setTaskContext(context) + val expectedSpillSize = map.getTotalMemoryConsumption + val sorter = new UnsafeKVExternalSorter( + schema, + schema, + sparkContext.env.blockManager, + sparkContext.env.serializerManager, + taskMemoryManager.pageSizeBytes(), + Int.MaxValue, + map) + assert(sorter.getSpillSize === expectedSpillSize) + } finally { + TaskContext.unset() + } + } + + test("SPARK-31952: UnsafeKVExternalSorter.merge should accumulate totalSpillBytes") { + val memoryManager = new TestMemoryManager(new SparkConf()) + val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + val map1 = createBytesToBytesMapWithDuplicateKeys(taskMemoryManager) + val map2 = createBytesToBytesMapWithDuplicateKeys(taskMemoryManager) + val schema = new StructType().add("i", IntegerType) + + try { + val context = new TaskContextImpl(0, 0, 0, 0, 0, taskMemoryManager, new Properties(), null) + TaskContext.setTaskContext(context) + val expectedSpillSize = map1.getTotalMemoryConsumption + map2.getTotalMemoryConsumption + val sorter1 = new UnsafeKVExternalSorter( + schema, + schema, + sparkContext.env.blockManager, + sparkContext.env.serializerManager, + taskMemoryManager.pageSizeBytes(), + Int.MaxValue, + map1) + val sorter2 = new UnsafeKVExternalSorter( + schema, + schema, + sparkContext.env.blockManager, + sparkContext.env.serializerManager, + taskMemoryManager.pageSizeBytes(), + Int.MaxValue, + map2) + sorter1.merge(sorter2) + assert(sorter1.getSpillSize === expectedSpillSize) + } finally { + TaskContext.unset() + } + } + + private def createBytesToBytesMapWithDuplicateKeys(taskMemoryManager: TaskMemoryManager) + : BytesToBytesMap = { + val map = new BytesToBytesMap(taskMemoryManager, 64, taskMemoryManager.pageSizeBytes()) + // Key/value are a unsafe rows with a single int column + val key = new UnsafeRow(1) + key.pointTo(new Array[Byte](32), 32) + key.setInt(0, 1) + val value = new UnsafeRow(1) + value.pointTo(new Array[Byte](32), 32) + value.setInt(0, 2) + for (_ <- 1 to 65) { + val loc = map.lookup(key.getBaseObject, key.getBaseOffset, key.getSizeInBytes) + loc.append( + key.getBaseObject, key.getBaseOffset, key.getSizeInBytes, + value.getBaseObject, value.getBaseOffset, value.getSizeInBytes) + } + map + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index eb5643df4c..71eaed269e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -71,28 +71,25 @@ class WholeStageCodegenSuite extends QueryTest with SharedSparkSession } test("ShuffledHashJoin should be included in WholeStageCodegen") { - withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "30", - SQLConf.SHUFFLE_PARTITIONS.key -> "2", - SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { - val df1 = spark.range(5).select($"id".as("k1")) - val df2 = spark.range(15).select($"id".as("k2")) - val df3 = spark.range(6).select($"id".as("k3")) - - // test one shuffled hash join - val oneJoinDF = df1.join(df2, $"k1" === $"k2") - assert(oneJoinDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true - }.size === 1) - checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) - - // test two shuffled hash joins - val twoJoinsDF = df1.join(df2, $"k1" === $"k2").join(df3, $"k1" === $"k3") - assert(twoJoinsDF.queryExecution.executedPlan.collect { - case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true - }.size === 2) - checkAnswer(twoJoinsDF, - Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) - } + val df1 = spark.range(5).select($"id".as("k1")) + val df2 = spark.range(15).select($"id".as("k2")) + val df3 = spark.range(6).select($"id".as("k3")) + + // test one shuffled hash join + val oneJoinDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") + assert(oneJoinDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 1) + checkAnswer(oneJoinDF, Seq(Row(0, 0), Row(1, 1), Row(2, 2), Row(3, 3), Row(4, 4))) + + // test two shuffled hash joins + val twoJoinsDF = df1.join(df2.hint("SHUFFLE_HASH"), $"k1" === $"k2") + .join(df3.hint("SHUFFLE_HASH"), $"k1" === $"k3") + assert(twoJoinsDF.queryExecution.executedPlan.collect { + case WholeStageCodegenExec(_ : ShuffledHashJoinExec) => true + }.size === 2) + checkAnswer(twoJoinsDF, + Seq(Row(0, 0, 0), Row(1, 1, 1), Row(2, 2, 2), Row(3, 3, 3), Row(4, 4, 4))) } test("Sort should be included in WholeStageCodegen") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala index f946a6779e..c6fe64d105 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/PassThroughEncodingSuite.scala @@ -46,7 +46,7 @@ class PassThroughSuite extends SparkFunSuite { val builder = TestCompressibleColumnBuilder(columnStats, columnType, PassThrough) - input.map { value => + input.foreach { value => val row = new GenericInternalRow(1) columnType.setField(row, 0, value) builder.appendFrom(row, 0) @@ -98,7 +98,7 @@ class PassThroughSuite extends SparkFunSuite { val row = new GenericInternalRow(1) val nullRow = new GenericInternalRow(1) nullRow.setNullAt(0) - input.map { value => + input.foreach { value => if (value == nullValue) { builder.appendFrom(nullRow, 0) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala index aadcda490b..942a3e8635 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionSuiteBase.scala @@ -39,6 +39,7 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil override val command = "ALTER TABLE .. DROP PARTITION" protected def notFullPartitionSpecErr: String + protected def nullPartitionValue: String protected def checkDropPartition( t: String, @@ -170,4 +171,14 @@ trait AlterTableDropPartitionSuiteBase extends QueryTest with DDLCommandTestUtil QueryTest.checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(1, 1))) } } + + test("SPARK-33591: null as a partition value") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 INT, p1 STRING) $defaultUsing PARTITIONED BY (p1)") + sql(s"ALTER TABLE $t ADD PARTITION (p1 = null)") + checkPartitions(t, Map("p1" -> nullPartitionValue)) + sql(s"ALTER TABLE $t DROP PARTITION (p1 = null)") + checkPartitions(t) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala index 58055262d3..7f66e28249 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionSuiteBase.scala @@ -163,4 +163,19 @@ trait AlterTableRenamePartitionSuiteBase extends QueryTest with DDLCommandTestUt } } } + + test("SPARK-34011: refresh cache after partition renaming") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + sql(s"INSERT INTO $t PARTITION (part=1) SELECT 1") + assert(!spark.catalog.isCached(t)) + sql(s"CACHE TABLE $t") + assert(spark.catalog.isCached(t)) + QueryTest.checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(0, 0), Row(1, 1))) + sql(s"ALTER TABLE $t PARTITION (part=0) RENAME TO PARTITION (part=2)") + assert(spark.catalog.isCached(t)) + QueryTest.checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(0, 2), Row(1, 1))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 4e2b67e532..946e8412cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -1277,36 +1277,6 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { assertUnsupported("ALTER VIEW dbx.tab1 DROP IF EXISTS PARTITION (b='2')") } - - test("show databases") { - sql("CREATE DATABASE showdb2B") - sql("CREATE DATABASE showdb1A") - - // check the result as well as its order - checkDataset(sql("SHOW DATABASES"), Row("default"), Row("showdb1a"), Row("showdb2b")) - - checkAnswer( - sql("SHOW DATABASES LIKE '*db1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES '*db1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE 'showdb1A'"), - Row("showdb1a") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE '*db1A|*db2B'"), - Row("showdb1a") :: - Row("showdb2b") :: Nil) - - checkAnswer( - sql("SHOW DATABASES LIKE 'non-existentdb'"), - Nil) - } - test("drop view - temporary view") { val catalog = spark.sessionState.catalog sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableParserSuite.scala index f88fff8ed3..60c7cd8dd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropTableParserSuite.scala @@ -30,26 +30,30 @@ class DropTableParserSuite extends AnalysisTest with SharedSparkSession { test("drop table") { parseCompare("DROP TABLE testcat.ns1.ns2.tbl", DropTable( - UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl"), "DROP TABLE"), + UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl"), "DROP TABLE", true), ifExists = false, purge = false)) parseCompare(s"DROP TABLE db.tab", DropTable( - UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = false, purge = false)) + UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE", true), + ifExists = false, + purge = false)) parseCompare(s"DROP TABLE IF EXISTS db.tab", DropTable( - UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE"), ifExists = true, purge = false)) + UnresolvedTableOrView(Seq("db", "tab"), "DROP TABLE", true), + ifExists = true, + purge = false)) parseCompare(s"DROP TABLE tab", DropTable( - UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = false)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE", true), ifExists = false, purge = false)) parseCompare(s"DROP TABLE IF EXISTS tab", DropTable( - UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = false)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE", true), ifExists = true, purge = false)) parseCompare(s"DROP TABLE tab PURGE", DropTable( - UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = false, purge = true)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE", true), ifExists = false, purge = true)) parseCompare(s"DROP TABLE IF EXISTS tab PURGE", DropTable( - UnresolvedTableOrView(Seq("tab"), "DROP TABLE"), ifExists = true, purge = true)) + UnresolvedTableOrView(Seq("tab"), "DROP TABLE", true), ifExists = true, purge = true)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala new file mode 100644 index 0000000000..c9e5d33fea --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesParserSuite.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.execution.command + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.ShowNamespaces +import org.apache.spark.sql.test.SharedSparkSession + +class ShowNamespacesParserSuite extends AnalysisTest with SharedSparkSession { + test("all namespaces") { + Seq("SHOW NAMESPACES", "SHOW DATABASES").foreach { sqlCmd => + comparePlans( + parsePlan(sqlCmd), + ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), None)) + } + } + + test("basic pattern") { + Seq( + "SHOW DATABASES LIKE 'defau*'", + "SHOW NAMESPACES LIKE 'defau*'").foreach { sqlCmd => + comparePlans( + parsePlan(sqlCmd), + ShowNamespaces(UnresolvedNamespace(Seq.empty[String]), Some("defau*"))) + } + } + + test("FROM/IN operator is not allowed by SHOW DATABASES") { + Seq( + "SHOW DATABASES FROM testcat.ns1.ns2", + "SHOW DATABASES IN testcat.ns1.ns2").foreach { sqlCmd => + val errMsg = intercept[ParseException] { + parsePlan(sqlCmd) + }.getMessage + assert(errMsg.contains("FROM/IN operator is not allowed in SHOW DATABASES")) + } + } + + test("show namespaces in/from a namespace") { + comparePlans( + parsePlan("SHOW NAMESPACES FROM testcat.ns1.ns2"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1.ns2"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1", "ns2")), None)) + } + + test("namespaces by a pattern from another namespace") { + comparePlans( + parsePlan("SHOW NAMESPACES IN testcat.ns1 LIKE '*pattern*'"), + ShowNamespaces(UnresolvedNamespace(Seq("testcat", "ns1")), Some("*pattern*"))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala new file mode 100644 index 0000000000..790489e0d4 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowNamespacesSuiteBase.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.execution.command + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StringType, StructType} + +/** + * This base suite contains unified tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands + * that check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are + * located in more specific test suites: + * + * - V2 table catalog tests: `org.apache.spark.sql.execution.command.v2.ShowNamespacesSuite` + * - V1 table catalog tests: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuiteBase` + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuite` + * - V1 Hive External catalog: `org.apache.spark.sql.hive.execution.command.ShowNamespacesSuite` + */ +trait ShowNamespacesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "SHOW NAMESPACES" + + protected def runShowNamespacesSql(sqlText: String, expected: Seq[String]): Unit = { + val df = spark.sql(sqlText) + assert(df.schema === new StructType().add("namespace", StringType, false)) + checkAnswer(df, expected.map(Row(_))) + } + + protected def builtinTopNamespaces: Seq[String] = Seq.empty + + test("default namespace") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES", builtinTopNamespaces) + } + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog", builtinTopNamespaces) + } + + test("at the top level") { + withNamespace(s"$catalog.ns1", s"$catalog.ns2") { + sql(s"CREATE DATABASE $catalog.ns1") + sql(s"CREATE NAMESPACE $catalog.ns2") + + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("ns1", "ns2") ++ builtinTopNamespaces) + } + } + + test("exact matching") { + withNamespace(s"$catalog.ns1", s"$catalog.ns2") { + sql(s"CREATE NAMESPACE $catalog.ns1") + sql(s"CREATE NAMESPACE $catalog.ns2") + Seq( + s"SHOW NAMESPACES IN $catalog LIKE 'ns2'", + s"SHOW NAMESPACES IN $catalog 'ns2'", + s"SHOW NAMESPACES FROM $catalog LIKE 'ns2'", + s"SHOW NAMESPACES FROM $catalog 'ns2'").foreach { sqlCmd => + withClue(sqlCmd) { + runShowNamespacesSql(sqlCmd, Seq("ns2")) + } + } + } + } + + test("does not match to any namespace") { + Seq( + "SHOW DATABASES LIKE 'non-existentdb'", + "SHOW NAMESPACES 'non-existentdb'").foreach { sqlCmd => + runShowNamespacesSql(sqlCmd, Seq.empty) + } + } + + test("show root namespaces with the default catalog") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES", builtinTopNamespaces) + + withNamespace("ns1", "ns2") { + sql(s"CREATE NAMESPACE ns1") + sql(s"CREATE NAMESPACE ns2") + + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns1", "ns2") ++ builtinTopNamespaces) + runShowNamespacesSql("SHOW NAMESPACES LIKE '*1*'", Seq("ns1")) + } + } + } + + test("complex namespace patterns") { + withNamespace(s"$catalog.showdb2b", s"$catalog.showdb1a") { + sql(s"CREATE NAMESPACE $catalog.showdb2b") + sql(s"CREATE NAMESPACE $catalog.showdb1a") + + Seq( + "'*db1A'" -> Seq("showdb1a"), + "'*2*'" -> Seq("showdb2b"), + "'*db1A|*db2B'" -> Seq("showdb1a", "showdb2b") + ).foreach { case (pattern, expected) => + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE $pattern", expected) + } + } + } + + test("change catalog and namespace with USE statements") { + try { + withNamespace(s"$catalog.ns") { + sql(s"CREATE NAMESPACE $catalog.ns") + sql(s"USE $catalog") + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns") ++ builtinTopNamespaces) + + sql("USE ns") + // 'SHOW NAMESPACES' is not affected by the current namespace and lists root namespaces. + runShowNamespacesSql("SHOW NAMESPACES", Seq("ns") ++ builtinTopNamespaces) + } + } finally { + spark.sessionState.catalogManager.reset() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala index b3c118def7..8f7b80c8eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableAddPartitionSuite.scala @@ -17,7 +17,11 @@ package org.apache.spark.sql.execution.command.v1 -import org.apache.spark.sql.AnalysisException +import java.io.File + +import org.apache.commons.io.FileUtils + +import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.execution.command /** @@ -41,6 +45,33 @@ trait AlterTableAddPartitionSuiteBase extends command.AlterTableAddPartitionSuit "The spec ([p1=]) contains an empty partition column value")) } } + + test("SPARK-34055: refresh cache in partition adding") { + withTable("t") { + sql(s"CREATE TABLE t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql("INSERT INTO t PARTITION (part=0) SELECT 0") + assert(!spark.catalog.isCached("t")) + sql("CACHE TABLE t") + assert(spark.catalog.isCached("t")) + checkAnswer(sql("SELECT * FROM t"), Seq(Row(0, 0))) + + // Create new partition (part = 1) in the filesystem + val information = sql("SHOW TABLE EXTENDED LIKE 't' PARTITION (part = 0)") + .select("information") + .first().getString(0) + val part0Loc = information + .split("\\r?\\n") + .filter(_.startsWith("Location:")) + .head + .replace("Location: file:", "") + val part1Loc = part0Loc.replace("part=0", "part=1") + FileUtils.copyDirectory(new File(part0Loc), new File(part1Loc)) + + sql(s"ALTER TABLE t ADD PARTITION (part=1) LOCATION '$part1Loc'") + assert(spark.catalog.isCached("t")) + checkAnswer(sql("SELECT * FROM t"), Seq(Row(0, 0), Row(0, 1))) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala index a6490ebdb9..509c0be28c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropPartitionSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.command */ trait AlterTableDropPartitionSuiteBase extends command.AlterTableDropPartitionSuiteBase { override protected val notFullPartitionSpecErr = "The following partitions not found in table" + override protected def nullPartitionValue: String = "__HIVE_DEFAULT_PARTITION__" test("purge partition data") { withNamespaceAndTable("ns", "tbl") { t => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala new file mode 100644 index 0000000000..fd76ef2490 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.execution.command.v1 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands + * that check V1 table catalogs. The tests that cannot run for all V1 catalogs are located in more + * specific test suites: + * + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.ShowNamespacesSuite` + * - V1 Hive External catalog: `org.apache.spark.sql.hive.execution.command.ShowNamespacesSuite` + */ +trait ShowNamespacesSuiteBase extends command.ShowNamespacesSuiteBase { + override protected def builtinTopNamespaces: Seq[String] = Seq("default") + + test("IN namespace doesn't exist") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES in dummy") + }.getMessage + assert(errMsg.contains("Namespace 'dummy' not found")) + } +} + +class ShowNamespacesSuite extends ShowNamespacesSuiteBase with CommandSuiteBase { + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + val expected = if (caseSensitive) "AAA" else "aaa" + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq(expected, "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq(expected)) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq(expected)) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala index e85d62c51e..a26e29706e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala @@ -69,6 +69,18 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase { assert(errMsg.contains("'SHOW PARTITIONS' expects a table")) } } + + test("SPARK-33591: null as a partition value") { + val t = "part_table" + withTable(t) { + sql(s"CREATE TABLE $t (col1 INT, p1 STRING) $defaultUsing PARTITIONED BY (p1)") + sql(s"INSERT INTO TABLE $t PARTITION (p1 = null) SELECT 0") + checkAnswer(sql(s"SHOW PARTITIONS $t"), Row("p1=__HIVE_DEFAULT_PARTITION__")) + checkAnswer( + sql(s"SHOW PARTITIONS $t PARTITION (p1 = null)"), + Row("p1=__HIVE_DEFAULT_PARTITION__")) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala index d6890d6fae..3515fa3390 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropPartitionSuite.scala @@ -27,8 +27,8 @@ import org.apache.spark.sql.execution.command class AlterTableDropPartitionSuite extends command.AlterTableDropPartitionSuiteBase with CommandSuiteBase { - override protected val notFullPartitionSpecErr = "Partition spec is invalid" + override protected def nullPartitionValue: String = "null" test("SPARK-33650: drop partition into a table which doesn't support partition management") { withNamespaceAndTable("ns", "tbl", s"non_part_$catalog") { t => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala new file mode 100644 index 0000000000..7a2c136eea --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/ShowNamespacesSuite.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.execution.command.v2 + +import org.apache.spark.SparkConf +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.connector.BasicInMemoryTableCatalog +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.SQLConf + +/** + * The class contains tests for the `SHOW NAMESPACES` command to check V2 table catalogs. + */ +class ShowNamespacesSuite extends command.ShowNamespacesSuiteBase with CommandSuiteBase { + override def sparkConf: SparkConf = super.sparkConf + .set("spark.sql.catalog.testcat_no_namespace", classOf[BasicInMemoryTableCatalog].getName) + + test("IN namespace doesn't exist") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> catalog) { + runShowNamespacesSql("SHOW NAMESPACES in dummy", Seq.empty) + } + runShowNamespacesSql(s"SHOW NAMESPACES in $catalog.ns1", Seq.empty) + runShowNamespacesSql(s"SHOW NAMESPACES in $catalog.ns1.ns3", Seq.empty) + } + + test("default v2 catalog doesn't support namespace") { + withSQLConf(SQLConf.DEFAULT_CATALOG.key -> "testcat_no_namespace") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES") + }.getMessage + assert(errMsg.contains("does not support namespaces")) + } + } + + test("v2 catalog doesn't support namespace") { + val errMsg = intercept[AnalysisException] { + sql("SHOW NAMESPACES in testcat_no_namespace") + }.getMessage + assert(errMsg.contains("does not support namespaces")) + } + + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("AAA", "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq("AAA")) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq("AAA")) + } + } + } + } +} diff --git a/sql/create-docs.sh b/sql/create-docs.sh index 6614c714e9..8721df874e 100755 --- a/sql/create-docs.sh +++ b/sql/create-docs.sh @@ -27,14 +27,14 @@ set -e FWDIR="$(cd "`dirname "${BASH_SOURCE[0]}"`"; pwd)" SPARK_HOME="$(cd "`dirname "${BASH_SOURCE[0]}"`"/..; pwd)" -if ! hash python 2>/dev/null; then - echo "Missing python in your path, skipping SQL documentation generation." +if ! hash python3 2>/dev/null; then + echo "Missing python3 in your path, skipping SQL documentation generation." exit 0 fi if ! hash mkdocs 2>/dev/null; then echo "Missing mkdocs in your path, trying to install mkdocs for SQL documentation generation." - pip install mkdocs + pip3 install mkdocs fi pushd "$FWDIR" > /dev/null diff --git a/sql/gen-sql-api-docs.py b/sql/gen-sql-api-docs.py index 61328997c1..2f734093b1 100644 --- a/sql/gen-sql-api-docs.py +++ b/sql/gen-sql-api-docs.py @@ -195,6 +195,7 @@ def generate_sql_api_markdown(jvm, path): """ with open(path, 'w') as mdfile: + mdfile.write("# Built-in Functions\n\n") for info in _list_function_infos(jvm): name = info.name usage = _make_pretty_usage(info.usage) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index f2fd373bf6..8606aaab1c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -522,15 +522,32 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // Note: [SPARK-31595] if there is a `'` in a double quoted string, or a `"` in a single quoted // string, the origin implementation from Hive will not drop the trailing semicolon as expected, // hence we refined this function a little bit. + // Note: [SPARK-33100] Ignore a semicolon inside a bracketed comment in spark-sql. private def splitSemiColon(line: String): JList[String] = { var insideSingleQuote = false var insideDoubleQuote = false - var insideComment = false + var insideSimpleComment = false + var bracketedCommentLevel = 0 var escape = false var beginIndex = 0 + var leavingBracketedComment = false + var isStatement = false val ret = new JArrayList[String] + def insideBracketedComment: Boolean = bracketedCommentLevel > 0 + def insideComment: Boolean = insideSimpleComment || insideBracketedComment + def statementInProgress(index: Int): Boolean = isStatement || (!insideComment && + index > beginIndex && !s"${line.charAt(index)}".trim.isEmpty) + for (index <- 0 until line.length) { + // Checks if we need to decrement a bracketed comment level; the last character '/' of + // bracketed comments is still inside the comment, so `insideBracketedComment` must keep true + // in the previous loop and we decrement the level here if needed. + if (leavingBracketedComment) { + bracketedCommentLevel -= 1 + leavingBracketedComment = false + } + if (line.charAt(index) == '\'' && !insideComment) { // take a look to see if it is escaped // See the comment above about SPARK-31595 @@ -553,21 +570,34 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { // Sample query: select "quoted value --" // ^^ avoids starting a comment if it's inside quotes. } else if (hasNext && line.charAt(index + 1) == '-') { - // ignore quotes and ; - insideComment = true + // ignore quotes and ; in simple comment + insideSimpleComment = true } } else if (line.charAt(index) == ';') { if (insideSingleQuote || insideDoubleQuote || insideComment) { // do not split } else { - // split, do not include ; itself - ret.add(line.substring(beginIndex, index)) + if (isStatement) { + // split, do not include ; itself + ret.add(line.substring(beginIndex, index)) + } beginIndex = index + 1 + isStatement = false } } else if (line.charAt(index) == '\n') { - // with a new line the inline comment should end. + // with a new line the inline simple comment should end. if (!escape) { - insideComment = false + insideSimpleComment = false + } + } else if (line.charAt(index) == '/' && !insideSimpleComment) { + val hasNext = index + 1 < line.length + if (insideSingleQuote || insideDoubleQuote) { + // Ignores '/' in any case of quotes + } else if (insideBracketedComment && line.charAt(index - 1) == '*' ) { + // Decrements `bracketedCommentLevel` at the beginning of the next loop + leavingBracketedComment = true + } else if (hasNext && !insideBracketedComment && line.charAt(index + 1) == '*') { + bracketedCommentLevel += 1 } } // set the escape @@ -576,8 +606,12 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } else if (line.charAt(index) == '\\') { escape = true } + + isStatement = statementInProgress(index) + } + if (isStatement) { + ret.add(line.substring(beginIndex)) } - ret.add(line.substring(beginIndex)) ret } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d39b94503f..1a96012a0b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -571,4 +571,27 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { // the date formatter for `java.sql.LocalDate` must output negative years with sign. runCliWithin(1.minute)("SELECT MAKE_DATE(-44, 3, 15);" -> "-0044-03-15") } + + test("SPARK-33100: Ignore a semicolon inside a bracketed comment in spark-sql") { + runCliWithin(4.minute)( + "/* SELECT 'test';*/ SELECT 'test';" -> "test", + ";;/* SELECT 'test';*/ SELECT 'test';" -> "test", + "/* SELECT 'test';*/;; SELECT 'test';" -> "test", + "SELECT 'test'; -- SELECT 'test';" -> "test", + "SELECT 'test'; /* SELECT 'test';*/;" -> "test", + "/*$meta chars{^\\;}*/ SELECT 'test';" -> "test", + "/*\nmulti-line\n*/ SELECT 'test';" -> "test", + "/*/* multi-level bracketed*/ SELECT 'test';" -> "test" + ) + } + + test("SPARK-33100: test sql statements with hint in bracketed comment") { + runCliWithin(2.minute)( + "CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES(1, 2) AS t1(k, v);" -> "", + "CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES(2, 1) AS t2(k, v);" -> "", + "EXPLAIN SELECT /*+ MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;" -> "SortMergeJoin", + "EXPLAIN SELECT /* + MERGEJOIN(t1) */ t1.* FROM t1 JOIN t2 ON t1.k = t2.v;" + -> "BroadcastHashJoin" + ) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala index b4aa073893..eeffe4f25d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -942,9 +942,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // Hive metastore is not case preserving and the partition columns are always lower cased. We need // to lower case the column names in partition specification before calling partition related Hive // APIs, to match this behaviour. - private def lowerCasePartitionSpec(spec: TablePartitionSpec): TablePartitionSpec = { + private def toMetaStorePartitionSpec(spec: TablePartitionSpec): TablePartitionSpec = { // scalastyle:off caselocale - spec.map { case (k, v) => k.toLowerCase -> v } + val lowNames = spec.map { case (k, v) => k.toLowerCase -> v } + ExternalCatalogUtils.convertNullPartitionValues(lowNames) // scalastyle:on caselocale } @@ -993,8 +994,9 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat } p.copy(storage = p.storage.copy(locationUri = Some(partitionPath.toUri))) } - val lowerCasedParts = partsWithLocation.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) - client.createPartitions(db, table, lowerCasedParts, ignoreIfExists) + val metaStoreParts = partsWithLocation + .map(p => p.copy(spec = toMetaStorePartitionSpec(p.spec))) + client.createPartitions(db, table, metaStoreParts, ignoreIfExists) } override def dropPartitions( @@ -1006,7 +1008,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat retainData: Boolean): Unit = withClient { requireTableExists(db, table) client.dropPartitions( - db, table, parts.map(lowerCasePartitionSpec), ignoreIfNotExists, purge, retainData) + db, table, parts.map(toMetaStorePartitionSpec), ignoreIfNotExists, purge, retainData) } override def renamePartitions( @@ -1015,7 +1017,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat specs: Seq[TablePartitionSpec], newSpecs: Seq[TablePartitionSpec]): Unit = withClient { client.renamePartitions( - db, table, specs.map(lowerCasePartitionSpec), newSpecs.map(lowerCasePartitionSpec)) + db, table, specs.map(toMetaStorePartitionSpec), newSpecs.map(toMetaStorePartitionSpec)) val tableMeta = getTable(db, table) val partitionColumnNames = tableMeta.partitionColumnNames @@ -1031,7 +1033,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val fs = tablePath.getFileSystem(hadoopConf) val newParts = newSpecs.map { spec => val rightPath = renamePartitionDirectory(fs, tablePath, partitionColumnNames, spec) - val partition = client.getPartition(db, table, lowerCasePartitionSpec(spec)) + val partition = client.getPartition(db, table, toMetaStorePartitionSpec(spec)) partition.copy(storage = partition.storage.copy(locationUri = Some(rightPath.toUri))) } alterPartitions(db, table, newParts) @@ -1141,12 +1143,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat db: String, table: String, newParts: Seq[CatalogTablePartition]): Unit = withClient { - val lowerCasedParts = newParts.map(p => p.copy(spec = lowerCasePartitionSpec(p.spec))) + val metaStoreParts = newParts.map(p => p.copy(spec = toMetaStorePartitionSpec(p.spec))) val rawTable = getRawTable(db, table) // convert partition statistics to properties so that we can persist them through hive api - val withStatsProps = lowerCasedParts.map { p => + val withStatsProps = metaStoreParts.map { p => if (p.stats.isDefined) { val statsProperties = statsToProperties(p.stats.get) p.copy(parameters = p.parameters ++ statsProperties) @@ -1162,7 +1164,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat db: String, table: String, spec: TablePartitionSpec): CatalogTablePartition = withClient { - val part = client.getPartition(db, table, lowerCasePartitionSpec(spec)) + val part = client.getPartition(db, table, toMetaStorePartitionSpec(spec)) restorePartitionMetadata(part, getTable(db, table)) } @@ -1200,7 +1202,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat db: String, table: String, spec: TablePartitionSpec): Option[CatalogTablePartition] = withClient { - client.getPartitionOption(db, table, lowerCasePartitionSpec(spec)).map { part => + client.getPartitionOption(db, table, toMetaStorePartitionSpec(spec)).map { part => restorePartitionMetadata(part, getTable(db, table)) } } @@ -1215,7 +1217,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat val catalogTable = getTable(db, table) val partColNameMap = buildLowerCasePartColNameMap(catalogTable).mapValues(escapePathName) val clientPartitionNames = - client.getPartitionNames(catalogTable, partialSpec.map(lowerCasePartitionSpec)) + client.getPartitionNames(catalogTable, partialSpec.map(toMetaStorePartitionSpec)) clientPartitionNames.map { partitionPath => val partSpec = PartitioningUtils.parsePathFragmentAsSeq(partitionPath) partSpec.map { case (partName, partValue) => @@ -1234,11 +1236,12 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat table: String, partialSpec: Option[TablePartitionSpec] = None): Seq[CatalogTablePartition] = withClient { val partColNameMap = buildLowerCasePartColNameMap(getTable(db, table)) - val res = client.getPartitions(db, table, partialSpec.map(lowerCasePartitionSpec)).map { part => - part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) + val metaStoreSpec = partialSpec.map(toMetaStorePartitionSpec) + val res = client.getPartitions(db, table, metaStoreSpec) + .map { part => part.copy(spec = restorePartitionSpec(part.spec, partColNameMap)) } - partialSpec match { + metaStoreSpec match { // This might be a bug of Hive: When the partition value inside the partial partition spec // contains dot, and we ask Hive to list partitions w.r.t. the partial partition spec, Hive // treats dot as matching any single character and may return more partitions than we diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index f60bad180a..a8439edcb8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -86,14 +86,7 @@ private[sql] class HiveSessionCatalog( } } catch { case NonFatal(e) => - val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" - val errorMsg = - if (classOf[GenericUDTF].isAssignableFrom(clazz)) { - s"$noHandlerMsg\nPlease make sure your function overrides " + - "`public StructObjectInspector initialize(ObjectInspector[] args)`." - } else { - noHandlerMsg - } + val errorMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" val analysisException = new AnalysisException(errorMsg) analysisException.setStackTrace(e.getStackTrace) throw analysisException diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 63e4688037..bfb24cfedb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -133,6 +133,7 @@ case class InsertIntoHiveTable( val numDynamicPartitions = partition.values.count(_.isEmpty) val numStaticPartitions = partition.values.count(_.nonEmpty) val partitionSpec = partition.map { + case (key, Some(null)) => key -> ExternalCatalogUtils.DEFAULT_PARTITION_NAME case (key, Some(value)) => key -> value case (key, None) => key -> "" } @@ -229,6 +230,7 @@ case class InsertIntoHiveTable( val caseInsensitiveDpMap = CaseInsensitiveMap(dpMap) val updatedPartitionSpec = partition.map { + case (key, Some(null)) => key -> ExternalCatalogUtils.DEFAULT_PARTITION_NAME case (key, Some(value)) => key -> value case (key, None) if caseInsensitiveDpMap.contains(key) => key -> caseInsensitiveDpMap(key) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index c7002853be..7717e6ee20 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -213,10 +213,14 @@ private[hive] case class HiveGenericUDTF( } @transient - protected lazy val inputInspectors = children.map(toInspector) + protected lazy val inputInspector = { + val inspectors = children.map(toInspector) + val fields = inspectors.indices.map(index => s"_col$index").asJava + ObjectInspectorFactory.getStandardStructObjectInspector(fields, inspectors.asJava) + } @transient - protected lazy val outputInspector = function.initialize(inputInspectors.toArray) + protected lazy val outputInspector = function.initialize(inputInspector) @transient protected lazy val udtInput = new Array[AnyRef](children.length) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala index dc56e6bc4d..fd83395bf4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HivePartitionFilteringSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.internal.SQLConf @@ -39,10 +40,13 @@ class HivePartitionFilteringSuite(version: String) private val tryDirectSqlKey = HiveConf.ConfVars.METASTORE_TRY_DIRECT_SQL.varname + // Support default partition in metastoredirectsql since HIVE-11898(Hive 2.0.0). + private val defaultPartition = if (version.toDouble >= 2) Some(DEFAULT_PARTITION_NAME) else None + private val dsValue = 20170101 to 20170103 private val hValue = 0 to 4 private val chunkValue = Seq("aa", "ab", "ba", "bb") - private val dateValue = Seq("2019-01-01", "2019-01-02", "2019-01-03") + private val dateValue = Seq("2019-01-01", "2019-01-02", "2019-01-03") ++ defaultPartition private val dateStrValue = Seq("2020-01-01", "2020-01-02", "2020-01-03") private val testPartitionCount = dsValue.size * hValue.size * chunkValue.size * dateValue.size * dateStrValue.size @@ -424,6 +428,71 @@ class HivePartitionFilteringSuite(version: String) dateStrValue) } + test("getPartitionsByFilter: IS NULL / IS NOT NULL") { + // returns all partitions + Seq(attr("d").isNull, attr("d").isNotNull).foreach { filterExpr => + testMetastorePartitionFiltering( + filterExpr, + dsValue, + hValue, + chunkValue, + dateValue, + dateStrValue) + } + } + + test("getPartitionsByFilter: IS NULL / IS NOT NULL with other filter") { + Seq(attr("d").isNull, attr("d").isNotNull).foreach { filterExpr => + testMetastorePartitionFiltering( + filterExpr && attr("d") === Date.valueOf("2019-01-01"), + dsValue, + hValue, + chunkValue, + Seq("2019-01-01"), + dateStrValue) + } + } + + test("getPartitionsByFilter: d =!= 2019-01-01") { + testMetastorePartitionFiltering( + attr("d") =!= Date.valueOf("2019-01-01"), + dsValue, + hValue, + chunkValue, + Seq("2019-01-02", "2019-01-03"), + dateStrValue) + } + + test("getPartitionsByFilter: d =!= 2019-01-01 || IS NULL") { + testMetastorePartitionFiltering( + attr("d") =!= Date.valueOf("2019-01-01") || attr("d").isNull, + dsValue, + hValue, + chunkValue, + dateValue, + dateStrValue) + } + + test("getPartitionsByFilter: d <=> 2019-01-01") { + testMetastorePartitionFiltering( + attr("d") <=> Date.valueOf("2019-01-01"), + dsValue, + hValue, + chunkValue, + dateValue, + dateStrValue) + } + + test("getPartitionsByFilter: d <=> null") { + testMetastorePartitionFiltering( + attr("d") <=> Literal(null, DateType), + dsValue, + hValue, + chunkValue, + dateValue, + dateStrValue) + } + private def testMetastorePartitionFiltering( filterExpr: Expression, expectedDs: Seq[Int], diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 3370695245..96c5bf7e27 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -2160,32 +2160,6 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } } - test("SPARK-21101 UDTF should override initialize(ObjectInspector[] args)") { - withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) { - sql( - s""" - |CREATE TEMPORARY FUNCTION udtf_stack1 - |AS 'org.apache.spark.sql.hive.execution.UDTFStack' - |USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}' - """.stripMargin) - val cnt = - sql("SELECT udtf_stack1(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')").count() - assert(cnt === 2) - - sql( - s""" - |CREATE TEMPORARY FUNCTION udtf_stack2 - |AS 'org.apache.spark.sql.hive.execution.UDTFStack2' - |USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}' - """.stripMargin) - val e = intercept[org.apache.spark.sql.AnalysisException] { - sql("SELECT udtf_stack2(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')") - } - assert( - e.getMessage.contains("public StructObjectInspector initialize(ObjectInspector[] args)")) - } - } - test("SPARK-21721: Clear FileSystem deleterOnExit cache if path is successfully removed") { val table = "test21721" withTable(table) { @@ -2583,6 +2557,30 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi } } } + + test("SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") { + withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) { + sql( + s""" + |CREATE TEMPORARY FUNCTION udtf_stack1 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack' + |USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}' + """.stripMargin) + sql( + s""" + |CREATE TEMPORARY FUNCTION udtf_stack2 + |AS 'org.apache.spark.sql.hive.execution.UDTFStack2' + |USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}' + """.stripMargin) + + Seq("udtf_stack1", "udtf_stack2").foreach { udf => + checkAnswer( + sql(s"SELECT $udf(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')"), + Seq(Row("A", 10, Date.valueOf("2015-01-01")), + Row("B", 20, Date.valueOf("2016-01-01")))) + } + } + } } @SlowHiveTest diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddPartitionSuite.scala index f8fe23f643..4363a617cf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableAddPartitionSuite.scala @@ -24,5 +24,21 @@ import org.apache.spark.sql.execution.command.v1 * V1 Hive external table catalog. */ class AlterTableAddPartitionSuite - extends v1.AlterTableAddPartitionSuiteBase - with CommandSuiteBase + extends v1.AlterTableAddPartitionSuiteBase + with CommandSuiteBase { + + test("hive client calls") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + + checkHiveClientCalls(expected = 20) { + sql(s"ALTER TABLE $t ADD PARTITION (part=1)") + } + sql(s"CACHE TABLE $t") + checkHiveClientCalls(expected = 23) { + sql(s"ALTER TABLE $t ADD PARTITION (part=2)") + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropPartitionSuite.scala index 5cac27f0d2..bbf8bde46e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropPartitionSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.sql.hive.execution.command +import org.apache.spark.sql.Row import org.apache.spark.sql.execution.command.v1 +import org.apache.spark.sql.internal.SQLConf /** * The class contains tests for the `ALTER TABLE .. DROP PARTITION` command to check @@ -25,4 +27,51 @@ import org.apache.spark.sql.execution.command.v1 */ class AlterTableDropPartitionSuite extends v1.AlterTableDropPartitionSuiteBase - with CommandSuiteBase + with CommandSuiteBase { + + test("hive client calls") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + sql(s"INSERT INTO $t PARTITION (part=1) SELECT 1") + + checkHiveClientCalls(expected = 19) { + sql(s"ALTER TABLE $t DROP PARTITION (part=0)") + } + sql(s"CACHE TABLE $t") + checkHiveClientCalls(expected = 22) { + sql(s"ALTER TABLE $t DROP PARTITION (part=1)") + } + } + } + + test("SPARK-34060: update stats of cached table") { + withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> "true") { + withNamespaceAndTable("ns", "tbl") { t => + def checkTableSize(expected: String): Unit = { + val stats = + sql(s"DESCRIBE TABLE EXTENDED $t") + .select("data_type") + .where("col_name = 'Statistics'") + .first() + .getString(0) + assert(stats.contains(expected)) + } + + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + sql(s"INSERT INTO $t PARTITION (part=1) SELECT 1") + assert(!spark.catalog.isCached(t)) + sql(s"CACHE TABLE $t") + assert(spark.catalog.isCached(t)) + checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(0, 0), Row(1, 1))) + checkTableSize("4 bytes") + + sql(s"ALTER TABLE $t DROP PARTITION (part=0)") + assert(spark.catalog.isCached(t)) + checkTableSize("2 bytes") + checkAnswer(sql(s"SELECT * FROM $t"), Seq(Row(1, 1))) + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenamePartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenamePartitionSuite.scala index 5cd5122a2a..413669a653 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenamePartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenamePartitionSuite.scala @@ -25,4 +25,20 @@ import org.apache.spark.sql.execution.command.v1 */ class AlterTableRenamePartitionSuite extends v1.AlterTableRenamePartitionSuiteBase - with CommandSuiteBase + with CommandSuiteBase { + + test("hive client calls") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, part int) $defaultUsing PARTITIONED BY (part)") + sql(s"INSERT INTO $t PARTITION (part=0) SELECT 0") + + checkHiveClientCalls(expected = 21) { + sql(s"ALTER TABLE $t PARTITION (part=0) RENAME TO PARTITION (part=1)") + } + sql(s"CACHE TABLE $t") + checkHiveClientCalls(expected = 24) { + sql(s"ALTER TABLE $t PARTITION (part=1) RENAME TO PARTITION (part=2)") + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CommandSuiteBase.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CommandSuiteBase.scala index a1c808647c..0709b12cdf 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CommandSuiteBase.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/CommandSuiteBase.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive.execution.command +import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.hive.test.TestHiveSingleton @@ -47,4 +48,11 @@ trait CommandSuiteBase extends TestHiveSingleton { val location = information.split("\\r?\\n").filter(_.startsWith("Location:")).head assert(location.endsWith(expected)) } + + def checkHiveClientCalls[T](expected: Int)(f: => T): Unit = { + HiveCatalogMetrics.reset() + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount === 0) + f + assert(HiveCatalogMetrics.METRIC_HIVE_CLIENT_CALLS.getCount === expected) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala index c1f17d1280..0ca6184c94 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala @@ -22,4 +22,13 @@ import org.apache.spark.sql.execution.command.v1 /** * The class contains tests for the `DROP TABLE` command to check V1 Hive external table catalog. */ -class DropTableSuite extends v1.DropTableSuiteBase with CommandSuiteBase +class DropTableSuite extends v1.DropTableSuiteBase with CommandSuiteBase { + test("hive client calls") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + checkHiveClientCalls(expected = 15) { + sql(s"DROP TABLE $t") + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala new file mode 100644 index 0000000000..7aae000144 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowNamespacesSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.spark.sql.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 +import org.apache.spark.sql.internal.SQLConf + +/** + * The class contains tests for the `SHOW NAMESPACES` and `SHOW DATABASES` commands to check + * V1 Hive external table catalog. + */ +class ShowNamespacesSuite extends v1.ShowNamespacesSuiteBase with CommandSuiteBase { + test("case sensitivity") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + withNamespace(s"$catalog.AAA", s"$catalog.bbb") { + sql(s"CREATE NAMESPACE $catalog.AAA") + sql(s"CREATE NAMESPACE $catalog.bbb") + runShowNamespacesSql( + s"SHOW NAMESPACES IN $catalog", + Seq("aaa", "bbb") ++ builtinTopNamespaces) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'AAA'", Seq("aaa")) + runShowNamespacesSql(s"SHOW NAMESPACES IN $catalog LIKE 'aaa'", Seq("aaa")) + } + } + } + } + + test("hive client calls") { + withNamespace(s"$catalog.ns1", s"$catalog.ns2") { + sql(s"CREATE NAMESPACE $catalog.ns1") + sql(s"CREATE NAMESPACE $catalog.ns2") + + checkHiveClientCalls(expected = 1) { + sql(s"SHOW NAMESPACES IN $catalog") + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowPartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowPartitionsSuite.scala index ded53cc3ea..c3d9790e1f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowPartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowPartitionsSuite.scala @@ -38,4 +38,13 @@ class ShowPartitionsSuite extends v1.ShowPartitionsSuiteBase with CommandSuiteBa } } } + + test("hive client calls") { + withNamespaceAndTable("ns", "dateTable") { t => + createDateTable(t) + checkHiveClientCalls(expected = 10) { + sql(s"SHOW PARTITIONS $t") + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala index 7b3652a860..b6db9a3e74 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/ShowTablesSuite.scala @@ -22,4 +22,13 @@ import org.apache.spark.sql.execution.command.v1 /** * The class contains tests for the `SHOW TABLES` command to check V1 Hive external table catalog. */ -class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase +class ShowTablesSuite extends v1.ShowTablesSuiteBase with CommandSuiteBase { + test("hive client calls") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + checkHiveClientCalls(expected = 3) { + sql(s"SHOW TABLES IN $catalog.ns") + } + } + } +}