diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index d82e1821ab140..c8b4c77d80615 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -79,10 +79,9 @@ jobs: INCLUDED_TAGS: ${{ matrix.included-tags }} HADOOP_PROFILE: ${{ matrix.hadoop }} HIVE_PROFILE: ${{ matrix.hive }} - # GitHub Actions' default miniconda to use in pip packaging test. - CONDA_PREFIX: /usr/share/miniconda GITHUB_PREV_SHA: ${{ github.event.before }} GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }} + SPARK_LOCAL_IP: localhost steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -166,10 +165,9 @@ jobs: MODULES_TO_TEST: ${{ matrix.modules }} HADOOP_PROFILE: hadoop3.2 HIVE_PROFILE: hive2.3 - # GitHub Actions' default miniconda to use in pip packaging test. - CONDA_PREFIX: /usr/share/miniconda GITHUB_PREV_SHA: ${{ github.event.before }} GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }} + SPARK_LOCAL_IP: localhost steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -211,9 +209,14 @@ jobs: run: | python3.6 -m pip install numpy 'pyarrow<3.0.0' pandas scipy xmlrunner python3.6 -m pip list + - name: Install Conda for pip packaging test + run: | + curl -s https://repo.anaconda.com/miniconda/Miniconda3-latest-Linux-x86_64.sh > miniconda.sh + bash miniconda.sh -b -p $HOME/miniconda # Run the tests. - name: Run tests run: | + export PATH=$PATH:$HOME/miniconda/bin ./dev/run-tests --parallelism 2 --modules "$MODULES_TO_TEST" - name: Upload test results to report if: always() @@ -238,6 +241,7 @@ jobs: HIVE_PROFILE: hive2.3 GITHUB_PREV_SHA: ${{ github.event.before }} GITHUB_INPUT_BRANCH: ${{ github.event.inputs.target }} + SPARK_LOCAL_IP: localhost steps: - name: Checkout Spark repository uses: actions/checkout@v2 @@ -285,6 +289,9 @@ jobs: lint: name: Linters, licenses, dependencies and documentation generation runs-on: ubuntu-20.04 + env: + LC_ALL: C.UTF-8 + LANG: C.UTF-8 container: image: dongjoon/apache-spark-github-action-image:20201025 steps: @@ -326,12 +333,14 @@ jobs: run: | # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. # See also https://github.com/sphinx-doc/sphinx/issues/7551. - python3.6 -m pip install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme ipython nbsphinx mypy numpydoc + # Jinja2 3.0.0+ causes error when building with Sphinx. + # See also https://issues.apache.org/jira/browse/SPARK-35375. + python3.6 -m pip install flake8 'sphinx<3.1.0' numpy pydata_sphinx_theme ipython nbsphinx mypy numpydoc 'jinja2<3.0.0' - name: Install R linter dependencies and SparkR run: | apt-get install -y libcurl4-openssl-dev libgit2-dev libssl-dev libxml2-dev Rscript -e "install.packages(c('devtools'), repos='https://cloud.r-project.org/')" - Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')" + Rscript -e "devtools::install_github('jimhester/lintr@v2.0.1')" ./R/install-dev.sh - name: Install dependencies for documentation generation run: | @@ -339,7 +348,9 @@ jobs: apt-get install -y libcurl4-openssl-dev pandoc # TODO(SPARK-32407): Sphinx 3.1+ does not correctly index nested classes. # See also https://github.com/sphinx-doc/sphinx/issues/7551. - python3.6 -m pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ipython nbsphinx numpydoc + # Jinja2 3.0.0+ causes error when building with Sphinx. + # See also https://issues.apache.org/jira/browse/SPARK-35375. + python3.6 -m pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ipython nbsphinx numpydoc 'jinja2<3.0.0' apt-get update -y apt-get install -y ruby ruby-dev gem install jekyll jekyll-redirect-from rouge @@ -359,8 +370,6 @@ jobs: - name: Run documentation build run: | cd docs - export LC_ALL=C.UTF-8 - export LANG=C.UTF-8 jekyll build java-11: @@ -369,6 +378,17 @@ jobs: steps: - name: Checkout Spark repository uses: actions/checkout@v2 + - name: Cache Scala, SBT and Maven + uses: actions/cache@v2 + with: + path: | + build/apache-maven-* + build/scala-* + build/*.jar + ~/.sbt + key: build-${{ hashFiles('**/pom.xml', 'project/build.properties', 'build/mvn', 'build/sbt', 'build/sbt-launch-lib.bash', 'build/spark-build-info') }} + restore-keys: | + build- - name: Cache Maven local repository uses: actions/cache@v2 with: @@ -388,45 +408,99 @@ jobs: ./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Djava.version=11 install rm -rf ~/.m2/repository/org/apache/spark - scala-213: - name: Scala 2.13 build with SBT + hadoop-2: + name: Hadoop 2 build with SBT runs-on: ubuntu-20.04 steps: - name: Checkout Spark repository uses: actions/checkout@v2 + - name: Cache Scala, SBT and Maven + uses: actions/cache@v2 + with: + path: | + build/apache-maven-* + build/scala-* + build/*.jar + ~/.sbt + key: build-${{ hashFiles('**/pom.xml', 'project/build.properties', 'build/mvn', 'build/sbt', 'build/sbt-launch-lib.bash', 'build/spark-build-info') }} + restore-keys: | + build- - name: Cache Coursier local repository uses: actions/cache@v2 with: path: ~/.cache/coursier - key: scala-213-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} + key: hadoop-2-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} restore-keys: | - scala-213-coursier- + hadoop-2-coursier- - name: Install Java 8 uses: actions/setup-java@v1 with: java-version: 8 - name: Build with SBT run: | - ./dev/change-scala-version.sh 2.13 - ./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Pdocker-integration-tests -Pkubernetes-integration-tests -Pspark-ganglia-lgpl -Pscala-2.13 compile test:compile + ./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Phadoop-2.7 compile test:compile - hadoop-2: - name: Hadoop 2 build with SBT + tpcds-1g: + name: Run TPC-DS queries with SF=1 runs-on: ubuntu-20.04 + env: + SPARK_LOCAL_IP: localhost steps: - name: Checkout Spark repository uses: actions/checkout@v2 + - name: Cache Scala, SBT and Maven + uses: actions/cache@v2 + with: + path: | + build/apache-maven-* + build/scala-* + build/*.jar + ~/.sbt + key: build-${{ hashFiles('**/pom.xml', 'project/build.properties', 'build/mvn', 'build/sbt', 'build/sbt-launch-lib.bash', 'build/spark-build-info') }} + restore-keys: | + build- - name: Cache Coursier local repository uses: actions/cache@v2 with: path: ~/.cache/coursier - key: hadoop-2-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} + key: tpcds-coursier-${{ hashFiles('**/pom.xml', '**/plugins.sbt') }} restore-keys: | - hadoop-2-coursier- + tpcds-coursier- - name: Install Java 8 uses: actions/setup-java@v1 with: java-version: 8 - - name: Build with SBT + - name: Cache TPC-DS generated data + id: cache-tpcds-sf-1 + uses: actions/cache@v2 + with: + path: ./tpcds-sf-1 + key: tpcds-${{ hashFiles('.github/workflows/build_and_test.yml', 'sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala') }} + - name: Checkout tpcds-kit repository + if: steps.cache-tpcds-sf-1.outputs.cache-hit != 'true' + uses: actions/checkout@v2 + with: + repository: databricks/tpcds-kit + ref: 2a5078a782192ddb6efbcead8de9973d6ab4f069 + path: ./tpcds-kit + - name: Build tpcds-kit + if: steps.cache-tpcds-sf-1.outputs.cache-hit != 'true' + run: cd tpcds-kit/tools && make OS=LINUX + - name: Generate TPC-DS (SF=1) table data + if: steps.cache-tpcds-sf-1.outputs.cache-hit != 'true' + run: build/sbt "sql/test:runMain org.apache.spark.sql.GenTPCDSData --dsdgenDir `pwd`/tpcds-kit/tools --location `pwd`/tpcds-sf-1 --scaleFactor 1 --numPartitions 1 --overwrite" + - name: Run TPC-DS queries run: | - ./build/sbt -Pyarn -Pmesos -Pkubernetes -Phive -Phive-thriftserver -Phadoop-cloud -Pkinesis-asl -Phadoop-2.7 compile test:compile + SPARK_TPCDS_DATA=`pwd`/tpcds-sf-1 build/sbt "sql/testOnly org.apache.spark.sql.TPCDSQueryTestSuite" + - name: Upload test results to report + if: always() + uses: actions/upload-artifact@v2 + with: + name: test-results-tpcds--8-hadoop3.2-hive2.3 + path: "**/target/test-reports/*.xml" + - name: Upload unit tests log files + if: failure() + uses: actions/upload-artifact@v2 + with: + name: unit-tests-log-tpcds--8-hadoop3.2-hive2.3 + path: "**/target/unit-tests.log" diff --git a/.github/workflows/test_report.yml b/.github/workflows/test_report.yml index 93cdb86687261..6d70f29b2efa2 100644 --- a/.github/workflows/test_report.yml +++ b/.github/workflows/test_report.yml @@ -15,6 +15,7 @@ jobs: github_token: ${{ secrets.GITHUB_TOKEN }} workflow: ${{ github.event.workflow_run.workflow_id }} commit: ${{ github.event.workflow_run.head_commit.id }} + workflow_conclusion: completed - name: Publish test report uses: scacap/action-surefire-report@v1 with: diff --git a/.sbtopts b/.sbtopts deleted file mode 100644 index 9afbdca6db1c7..0000000000000 --- a/.sbtopts +++ /dev/null @@ -1,17 +0,0 @@ -# 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. - --J-Xmx4G --J-Xss4m diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md index fd0c7644189db..d4ff3f5f4875b 100644 --- a/R/DOCUMENTATION.md +++ b/R/DOCUMENTATION.md @@ -19,7 +19,7 @@ license: | # SparkR Documentation SparkR documentation is generated by using in-source comments and annotated by using -[`roxygen2`](https://cran.r-project.org/web/packages/roxygen2/index.html). After making changes to the documentation and generating man pages, +[`roxygen2`](https://cran.r-project.org/package=roxygen2). After making changes to the documentation and generating man pages, you can run the following from an R console in the SparkR home directory ```R library(devtools) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 04ebb0ce8c571..da1b542d15f81 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,13 +1,13 @@ Package: SparkR Type: Package -Version: 3.1.1 +Version: 3.1.2 Title: R Front End for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . -Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), +Authors@R: c(person("Shivaram", "Venkataraman", role = "aut", email = "shivaram@cs.berkeley.edu"), person("Xiangrui", "Meng", role = "aut", email = "meng@databricks.com"), - person("Felix", "Cheung", role = "aut", + person("Felix", "Cheung", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) License: Apache License (== 2.0) @@ -20,6 +20,7 @@ Depends: Suggests: knitr, rmarkdown, + markdown, testthat, e1071, survival, diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 43b25a14e79ba..28e4ef874e77f 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -3578,7 +3578,12 @@ unresolved_named_lambda_var <- function(...) { "org.apache.spark.sql.Column", newJObject( "org.apache.spark.sql.catalyst.expressions.UnresolvedNamedLambdaVariable", - list(...) + lapply(list(...), function(x) { + handledCallJStatic( + "org.apache.spark.sql.catalyst.expressions.UnresolvedNamedLambdaVariable", + "freshVarName", + x) + }) ) ) column(jc) diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R index 71ebe4e26ef63..093467ecf7d28 100644 --- a/R/pkg/R/mllib_classification.R +++ b/R/pkg/R/mllib_classification.R @@ -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/web/packages/e1071/index.html} +#' @seealso e1071: \url{https://cran.r-project.org/package=e1071} #' @examples #' \dontrun{ #' data <- as.data.frame(UCBAdmissions) diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index ff7cbd8fc9b74..8bc15353465d8 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/web/packages/mixtools/index.html} +#' @seealso mixtools: \url{https://cran.r-project.org/package=mixtools} #' @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/web/packages/topicmodels/index.html} +#' @seealso topicmodels: \url{https://cran.r-project.org/package=topicmodels} #' @examples #' \dontrun{ #' text <- read.df("data/mllib/sample_lda_libsvm_data.txt", source = "libsvm") diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R index db9f367407df3..b2228a141689b 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/web/packages/survival/index.html} +#' @seealso survival: \url{https://cran.r-project.org/package=survival} #' @examples #' \dontrun{ #' df <- createDataFrame(ovarian) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index ebf08b9559379..23268974fc922 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -2153,6 +2153,20 @@ test_that("higher order functions", { expect_error(array_transform("xs", function(...) 42)) }) +test_that("SPARK-34794: lambda vars must be resolved properly in nested higher order functions", { + df <- sql("SELECT array(1, 2, 3) as numbers, array('a', 'b', 'c') as letters") + ret <- first(select( + df, + array_transform("numbers", function(number) { + array_transform("letters", function(latter) { + struct(alias(number, "n"), alias(latter, "l")) + }) + }) + )) + + expect_equal(1, ret[[1]][[1]][[1]][[1]]$n) +}) + test_that("group by, agg functions", { df <- read.json(jsonPath) df1 <- agg(df, name = "max", age = "sum") diff --git a/assembly/pom.xml b/assembly/pom.xml index 322f696a71350..6263ee4932d51 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/build/mvn b/build/mvn index 53a77f6e81f2b..2c8571f491757 100755 --- a/build/mvn +++ b/build/mvn @@ -26,36 +26,71 @@ _COMPILE_JVM_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g" # Installs any application tarball given a URL, the expected tarball name, # and, optionally, a checkable binary path to determine if the binary has -# already been installed -## Arg1 - URL -## Arg2 - Tarball Name -## Arg3 - Checkable Binary +# already been installed. Arguments: +# 1 - Mirror host +# 2 - URL path on host +# 3 - URL query string +# 4 - checksum suffix +# 5 - Tarball Name +# 6 - Checkable Binary install_app() { - local remote_tarball="$1/$2" - local local_tarball="${_DIR}/$2" - local binary="${_DIR}/$3" + local mirror_host="$1" + local url_path="$2" + local url_query="$3" + local checksum_suffix="$4" + local local_tarball="${_DIR}/$5" + local binary="${_DIR}/$6" + local remote_tarball="${mirror_host}/${url_path}${url_query}" + local local_checksum="${local_tarball}.${checksum_suffix}" + local remote_checksum="https://archive.apache.org/dist/${url_path}.${checksum_suffix}" local curl_opts="--silent --show-error -L" local wget_opts="--no-verbose" - if [ -z "$3" -o ! -f "$binary" ]; then + if [ ! -f "$binary" ]; then # check if we already have the tarball # check if we have curl installed # download application - [ ! -f "${local_tarball}" ] && [ $(command -v curl) ] && \ - echo "exec: curl ${curl_opts} ${remote_tarball}" 1>&2 && \ + if [ ! -f "${local_tarball}" -a "$(command -v curl)" ]; then + echo "exec: curl ${curl_opts} ${remote_tarball}" 1>&2 curl ${curl_opts} "${remote_tarball}" > "${local_tarball}" + if [ ! -z "${checksum_suffix}" ]; then + echo "exec: curl ${curl_opts} ${remote_checksum}" 1>&2 + curl ${curl_opts} "${remote_checksum}" > "${local_checksum}" + fi + fi # if the file still doesn't exist, lets try `wget` and cross our fingers - [ ! -f "${local_tarball}" ] && [ $(command -v wget) ] && \ - echo "exec: wget ${wget_opts} ${remote_tarball}" 1>&2 && \ + if [ ! -f "${local_tarball}" -a "$(command -v wget)" ]; then + echo "exec: wget ${wget_opts} ${remote_tarball}" 1>&2 wget ${wget_opts} -O "${local_tarball}" "${remote_tarball}" + if [ ! -z "${checksum_suffix}" ]; then + echo "exec: wget ${wget_opts} ${remote_checksum}" 1>&2 + wget ${wget_opts} -O "${local_checksum}" "${remote_checksum}" + fi + fi # if both were unsuccessful, exit - [ ! -f "${local_tarball}" ] && \ - echo -n "ERROR: Cannot download $2 with cURL or wget; " && \ - echo "please install manually and try again." && \ + if [ ! -f "${local_tarball}" ]; then + echo -n "ERROR: Cannot download ${remote_tarball} with cURL or wget; please install manually and try again." exit 2 - cd "${_DIR}" && tar -xzf "$2" - rm -rf "$local_tarball" + fi + # Checksum may not have been specified; don't check if doesn't exist + if [ "$(command -v shasum)" ]; then + if [ -f "${local_checksum}" ]; then + echo " ${local_tarball}" >> ${local_checksum} # two spaces + file are important! + # Assuming SHA512 here for now + echo "Veryfing checksum from ${local_checksum}" 1>&2 + if ! shasum -a 512 -c "${local_checksum}" > /dev/null ; then + echo "Bad checksum from ${remote_checksum}" + exit 2 + fi + fi + else + echo "Skipping checksum because shasum is not installed." 1>&2 + fi + + cd "${_DIR}" && tar -xzf "${local_tarball}" + rm -rf "${local_tarball}" + rm -f "${local_checksum}" fi } @@ -71,20 +106,26 @@ install_mvn() { local MVN_DETECTED_VERSION="$(mvn --version | head -n1 | awk '{print $3}')" fi if [ $(version $MVN_DETECTED_VERSION) -lt $(version $MVN_VERSION) ]; then - local APACHE_MIRROR=${APACHE_MIRROR:-'https://www.apache.org/dyn/closer.lua?action=download&filename='} - + local MVN_TARBALL="apache-maven-${MVN_VERSION}-bin.tar.gz" + local FILE_PATH="maven/maven-3/${MVN_VERSION}/binaries/${MVN_TARBALL}" + local APACHE_MIRROR=${APACHE_MIRROR:-'https://www.apache.org/dyn/closer.lua'} + local MIRROR_URL_QUERY="?action=download" + if [ $(command -v curl) ]; then - local TEST_MIRROR_URL="${APACHE_MIRROR}/maven/maven-3/${MVN_VERSION}/binaries/apache-maven-${MVN_VERSION}-bin.tar.gz" - if ! curl -L --output /dev/null --silent --head --fail "$TEST_MIRROR_URL" ; then + if ! curl -L --output /dev/null --silent --head --fail "${APACHE_MIRROR}/${FILE_PATH}${MIRROR_URL_QUERY}" ; then # Fall back to archive.apache.org for older Maven echo "Falling back to archive.apache.org to download Maven" APACHE_MIRROR="https://archive.apache.org/dist" + MIRROR_URL_QUERY="" fi fi install_app \ - "${APACHE_MIRROR}/maven/maven-3/${MVN_VERSION}/binaries" \ - "apache-maven-${MVN_VERSION}-bin.tar.gz" \ + "${APACHE_MIRROR}" \ + "${FILE_PATH}" \ + "${MIRROR_URL_QUERY}" \ + "sha512" \ + "${MVN_TARBALL}" \ "apache-maven-${MVN_VERSION}/bin/mvn" MVN_BIN="${_DIR}/apache-maven-${MVN_VERSION}/bin/mvn" @@ -105,7 +146,10 @@ install_zinc() { local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.lightbend.com} install_app \ - "${TYPESAFE_MIRROR}/zinc/${ZINC_VERSION}" \ + "${TYPESAFE_MIRROR}" \ + "zinc/${ZINC_VERSION}/zinc-${ZINC_VERSION}.tgz" \ + "" \ + "" \ "zinc-${ZINC_VERSION}.tgz" \ "${zinc_path}" ZINC_BIN="${_DIR}/${zinc_path}" @@ -121,10 +165,14 @@ install_scala() { local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | grep ${scala_binary_version} | head -n1 | awk -F '[<>]' '{print $3}'` local scala_bin="${_DIR}/scala-${scala_version}/bin/scala" local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.lightbend.com} + local SCALA_TARBALL="scala-${scala_version}.tgz" install_app \ - "${TYPESAFE_MIRROR}/scala/${scala_version}" \ - "scala-${scala_version}.tgz" \ + "${TYPESAFE_MIRROR}" \ + "scala/${scala_version}/${SCALA_TARBALL}" \ + "" \ + "" \ + ${SCALA_TARBALL} \ "scala-${scala_version}/bin/scala" SCALA_COMPILER="$(cd "$(dirname "${scala_bin}")/../lib" && pwd)/scala-compiler.jar" diff --git a/build/sbt b/build/sbt index 475dfd3b20b43..ae9ca93fc9ca9 100755 --- a/build/sbt +++ b/build/sbt @@ -53,6 +53,7 @@ realpath () { declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" declare -r sbt_opts_file=".sbtopts" declare -r etc_sbt_opts_file="/etc/sbt/sbtopts" +declare -r default_sbt_opts="-Xss4m" usage() { cat < org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 49396eea3e6f2..85a4631729dc1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 2f143f77fa4ae..3aac2d2441d2a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -188,6 +188,7 @@ public void handle(ResponseMessage message) throws Exception { if (listener == null) { logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", resp.requestId, getRemoteAddress(channel), resp.body().size()); + resp.body().release(); } else { outstandingRpcs.remove(resp.requestId); try { diff --git a/common/network-common/src/test/java/org/apache/spark/network/TestUtils.java b/common/network-common/src/test/java/org/apache/spark/network/TestUtils.java index 56a2b805f154c..c2c5ffa43e0ed 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/TestUtils.java +++ b/common/network-common/src/test/java/org/apache/spark/network/TestUtils.java @@ -22,7 +22,9 @@ public class TestUtils { public static String getLocalHost() { try { - return InetAddress.getLocalHost().getHostAddress(); + return (System.getenv().containsKey("SPARK_LOCAL_IP"))? + System.getenv("SPARK_LOCAL_IP"): + InetAddress.getLocalHost().getHostAddress(); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index c349871ada4f9..bca0537e4d7b4 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index ec2e3dce661d9..0b7eaa6225a41 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -21,7 +21,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.HashMap; +import java.util.LinkedHashMap; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; @@ -81,7 +81,6 @@ public OneForOneBlockFetcher( TransportConf transportConf, DownloadFileManager downloadFileManager) { this.client = client; - this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); this.transportConf = transportConf; @@ -90,8 +89,10 @@ public OneForOneBlockFetcher( throw new IllegalArgumentException("Zero-sized blockIds array"); } if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { - this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); + this.blockIds = new String[blockIds.length]; + this.message = createFetchShuffleBlocksMsgAndBuildBlockIds(appId, execId, blockIds); } else { + this.blockIds = blockIds; this.message = new OpenBlocks(appId, execId, blockIds); } } @@ -106,17 +107,16 @@ private boolean isShuffleBlocks(String[] blockIds) { } /** - * Analyze the pass in blockIds and create FetchShuffleBlocks message. - * The blockIds has been sorted by mapId and reduceId. It's produced in - * org.apache.spark.MapOutputTracker.convertMapStatuses. + * Create FetchShuffleBlocks message and rebuild internal blockIds by + * analyzing the pass in blockIds. */ - private FetchShuffleBlocks createFetchShuffleBlocksMsg( + private FetchShuffleBlocks createFetchShuffleBlocksMsgAndBuildBlockIds( String appId, String execId, String[] blockIds) { String[] firstBlock = splitBlockId(blockIds[0]); int shuffleId = Integer.parseInt(firstBlock[1]); boolean batchFetchEnabled = firstBlock.length == 5; - HashMap> mapIdToReduceIds = new HashMap<>(); + LinkedHashMap mapIdToBlocksInfo = new LinkedHashMap<>(); for (String blockId : blockIds) { String[] blockIdParts = splitBlockId(blockId); if (Integer.parseInt(blockIdParts[1]) != shuffleId) { @@ -124,23 +124,36 @@ private FetchShuffleBlocks createFetchShuffleBlocksMsg( ", got:" + blockId); } long mapId = Long.parseLong(blockIdParts[2]); - if (!mapIdToReduceIds.containsKey(mapId)) { - mapIdToReduceIds.put(mapId, new ArrayList<>()); + if (!mapIdToBlocksInfo.containsKey(mapId)) { + mapIdToBlocksInfo.put(mapId, new BlocksInfo()); } - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[3])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapId); + blocksInfoByMapId.blockIds.add(blockId); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[3])); if (batchFetchEnabled) { // When we read continuous shuffle blocks in batch, we will reuse reduceIds in // FetchShuffleBlocks to store the start and end reduce id for range // [startReduceId, endReduceId). assert(blockIdParts.length == 5); - mapIdToReduceIds.get(mapId).add(Integer.parseInt(blockIdParts[4])); + blocksInfoByMapId.reduceIds.add(Integer.parseInt(blockIdParts[4])); } } - long[] mapIds = Longs.toArray(mapIdToReduceIds.keySet()); + long[] mapIds = Longs.toArray(mapIdToBlocksInfo.keySet()); int[][] reduceIdArr = new int[mapIds.length][]; + int blockIdIndex = 0; for (int i = 0; i < mapIds.length; i++) { - reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); + BlocksInfo blocksInfoByMapId = mapIdToBlocksInfo.get(mapIds[i]); + reduceIdArr[i] = Ints.toArray(blocksInfoByMapId.reduceIds); + + // The `blockIds`'s order must be same with the read order specified in in FetchShuffleBlocks + // because the shuffle data's return order should match the `blockIds`'s order to ensure + // blockId and data match. + for (int j = 0; j < blocksInfoByMapId.blockIds.size(); j++) { + this.blockIds[blockIdIndex++] = blocksInfoByMapId.blockIds.get(j); + } } + assert(blockIdIndex == this.blockIds.length); + return new FetchShuffleBlocks( appId, execId, shuffleId, mapIds, reduceIdArr, batchFetchEnabled); } @@ -157,6 +170,18 @@ private String[] splitBlockId(String blockId) { return blockIdParts; } + /** The reduceIds and blocks in a single mapId */ + private class BlocksInfo { + + final ArrayList reduceIds; + final ArrayList blockIds; + + BlocksInfo() { + this.reduceIds = new ArrayList<>(); + this.blockIds = new ArrayList<>(); + } + } + /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ private class ChunkCallback implements ChunkReceivedCallback { @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 9b01f6dac5ba5..0b9864d42957b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -30,7 +30,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; @@ -446,9 +445,9 @@ static class PushBlockStreamCallback implements StreamCallbackWithID { private final AppShufflePartitionInfo partitionInfo; private int length = 0; // This indicates that this stream got the opportunity to write the blocks to the merged file. - // Once this is set to true and the stream encounters a failure then it will take necessary - // action to overwrite any partial written data. This is reset to false when the stream - // completes without any failures. + // Once this is set to true and the stream encounters a failure then it will unset the + // currentMapId of the partition so that another stream can start merging the blocks to the + // partition. This is reset to false when the stream completes. private boolean isWriting = false; // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly private List deferredBufs; @@ -478,16 +477,11 @@ public String getID() { */ private void writeBuf(ByteBuffer buf) throws IOException { while (buf.hasRemaining()) { - if (partitionInfo.isEncounteredFailure()) { - long updatedPos = partitionInfo.getDataFilePos() + length; - logger.debug( - "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}", - partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, - partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos); - length += partitionInfo.dataChannel.write(buf, updatedPos); - } else { - length += partitionInfo.dataChannel.write(buf); - } + long updatedPos = partitionInfo.getDataFilePos() + length; + logger.debug("{} shuffleId {} reduceId {} current pos {} updated pos {}", + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId, partitionInfo.getDataFilePos(), updatedPos); + length += partitionInfo.dataChannel.write(buf, updatedPos); } } @@ -582,7 +576,6 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { } // Check whether we can write to disk if (allowedToWrite()) { - isWriting = true; // Identify duplicate block generated by speculative tasks. We respond success to // the client in cases of duplicate even though no data is written. if (isDuplicateBlock()) { @@ -599,6 +592,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // If we got here, it's safe to write the block data to the merged shuffle file. We // first write any deferred block. + isWriting = true; try { if (deferredBufs != null && !deferredBufs.isEmpty()) { writeDeferredBufs(); @@ -610,16 +604,6 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // back to the client so the block could be retried. throw ioe; } - // If we got here, it means we successfully write the current chunk of block to merged - // shuffle file. If we encountered failure while writing the previous block, we should - // reset the file channel position and the status of partitionInfo to indicate that we - // have recovered from previous disk write failure. However, we do not update the - // position tracked by partitionInfo here. That is only updated while the entire block - // is successfully written to merged shuffle file. - if (partitionInfo.isEncounteredFailure()) { - partitionInfo.dataChannel.position(partitionInfo.getDataFilePos() + length); - partitionInfo.setEncounteredFailure(false); - } } else { logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, @@ -640,7 +624,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // written to disk due to this reason. We thus decide to optimize for server // throughput and memory usage. if (deferredBufs == null) { - deferredBufs = new LinkedList<>(); + deferredBufs = new ArrayList<>(); } // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger // byte buffer, we cache only the relevant bytes not the entire large buffer to save @@ -671,7 +655,6 @@ public void onComplete(String streamId) throws IOException { } // Check if we can commit this block if (allowedToWrite()) { - isWriting = true; // Identify duplicate block generated by speculative tasks. We respond success to // the client in cases of duplicate even though no data is written. if (isDuplicateBlock()) { @@ -682,6 +665,7 @@ public void onComplete(String streamId) throws IOException { try { if (deferredBufs != null && !deferredBufs.isEmpty()) { abortIfNecessary(); + isWriting = true; writeDeferredBufs(); } } catch (IOException ioe) { @@ -739,14 +723,14 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { Map shufflePartitions = mergeManager.partitions.get(partitionInfo.appShuffleId); if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) { - logger.debug("{} shuffleId {} reduceId {} set encountered failure", + logger.debug("{} shuffleId {} reduceId {} encountered failure", partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId); partitionInfo.setCurrentMapIndex(-1); - partitionInfo.setEncounteredFailure(true); } } } + isWriting = false; } @VisibleForTesting @@ -803,8 +787,6 @@ public static class AppShufflePartitionInfo { public FileChannel dataChannel; // Location offset of the last successfully merged block for this shuffle partition private long dataFilePos; - // Indicating whether failure was encountered when merging the previous block - private boolean encounteredFailure; // Track the map index whose block is being merged for this shuffle partition private int currentMapIndex; // Bitmap tracking which mapper's blocks have been merged for this shuffle partition @@ -837,7 +819,6 @@ public static class AppShufflePartitionInfo { // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() updateChunkInfo(0L, -1); this.dataFilePos = 0; - this.encounteredFailure = false; this.mapTracker = new RoaringBitmap(); this.chunkTracker = new RoaringBitmap(); } @@ -852,14 +833,6 @@ public void setDataFilePos(long dataFilePos) { this.dataFilePos = dataFilePos; } - boolean isEncounteredFailure() { - return encounteredFailure; - } - - void setEncounteredFailure(boolean encounteredFailure) { - this.encounteredFailure = encounteredFailure; - } - int getCurrentMapIndex() { return currentMapIndex; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index 285eedb39c65c..a7eb59d366966 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -201,6 +201,48 @@ public void testEmptyBlockFetch() { } } + @Test + public void testFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{0}, {1}, {2}}, false), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + + @Test + public void testBatchFetchShuffleBlocksOrder() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_1_2", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shuffle_0_2_2_3", new NioManagedBuffer(ByteBuffer.wrap(new byte[2]))); + blocks.put("shuffle_0_10_3_4", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[3]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, + new long[]{0, 2, 10}, new int[][]{{1, 2}, {2, 3}, {3, 4}}, true), + conf); + + for (int chunkIndex = 0; chunkIndex < blockIds.length; chunkIndex++) { + String blockId = blockIds[chunkIndex]; + verify(listener).onBlockFetchSuccess(blockId, blocks.get(blockId)); + } + } + /** * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which * simply returns the given (BlockId, Block) pairs. diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 8c6f7434748ec..565d433ff3203 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -28,6 +28,7 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadLocalRandom; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; @@ -292,18 +293,32 @@ public void testTooLateArrival() throws IOException { @Test public void testIncompleteStreamsAreOverwritten() throws IOException { registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); + byte[] expectedBytes = new byte[4]; + ThreadLocalRandom.current().nextBytes(expectedBytes); + StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); - stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4])); + byte[] data = new byte[10]; + ThreadLocalRandom.current().nextBytes(data); + stream1.onData(stream1.getID(), ByteBuffer.wrap(data)); // There is a failure stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); StreamCallbackWithID stream2 = pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); - stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + ByteBuffer nextBuf= ByteBuffer.wrap(expectedBytes, 0, 2); + stream2.onData(stream2.getID(), nextBuf); stream2.onComplete(stream2.getID()); + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); + nextBuf = ByteBuffer.wrap(expectedBytes, 2, 2); + stream3.onData(stream3.getID(), nextBuf); + stream3.onComplete(stream3.getID()); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); - validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5}, new int[][]{{1}}); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{1, 2}}); + FileSegmentManagedBuffer mb = + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, 0, 0, 0); + assertArrayEquals(expectedBytes, mb.nioByteBuffer().array()); } @Test (expected = RuntimeException.class) @@ -740,6 +755,72 @@ public void testFailureWhileTruncatingFiles() throws IOException { validateChunks(TEST_APP, 0, 1, meta, new int[]{5, 3}, new int[][]{{0},{1}}); } + @Test + public void testOnFailureInvokedMoreThanOncePerBlock() throws IOException { + StreamCallbackWithID stream1 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + stream1.onFailure(stream1.getID(), new RuntimeException("forced error")); + StreamCallbackWithID stream2 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + // On failure on stream1 gets invoked again and should cause no interference + stream1.onFailure(stream1.getID(), new RuntimeException("2nd forced error")); + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 3, 0, 0)); + // This should be deferred as stream 2 is still the active stream + stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); + // Stream 2 writes more and completes + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); + stream2.onComplete(stream2.getID()); + stream3.onComplete(stream3.getID()); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {9, 2}, new int[][] {{1},{3}}); + removeApplication(TEST_APP); + } + + @Test (expected = RuntimeException.class) + public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws IOException { + StreamCallbackWithID stream1 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + StreamCallbackWithID stream1Duplicate = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0)); + stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + stream1.onComplete(stream1.getID()); + stream1Duplicate.onData(stream1.getID(), ByteBuffer.wrap(new byte[2])); + + StreamCallbackWithID stream2 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5])); + // Should not change the current map id of the reduce partition + stream1Duplicate.onFailure(stream2.getID(), new RuntimeException("forced error")); + + StreamCallbackWithID stream3 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0)); + // This should be deferred as stream 2 is still the active stream + stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[2])); + RuntimeException failedEx = null; + try { + stream3.onComplete(stream3.getID()); + } catch (RuntimeException re) { + assertEquals( + "Couldn't find an opportunity to write block shufflePush_0_2_0 to merged shuffle", + re.getMessage()); + failedEx = re; + } + // Stream 2 writes more and completes + stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[4])); + stream2.onComplete(stream2.getID()); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {11}, new int[][] {{0, 1}}); + removeApplication(TEST_APP); + if (failedEx != null) { + throw failedEx; + } + } + private void useTestFiles(boolean useTestIndexFile, boolean useTestMetaFile) throws IOException { pushResolver = new RemoteBlockPushResolver(conf) { @Override diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 97b0b04f55c65..a0b7e88e6c935 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 6315452790059..613561b92a954 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 47aaccd9575a9..c749387db4975 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index b11fc8077fbbd..e0d48291afa2f 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index e91595dd324b0..dc7b9ea2d2ba1 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -38,3 +38,9 @@ log4j.logger.parquet=ERROR # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR + +# For deploying Spark ThriftServer +# SPARK-34128:Suppress undesirable TTransportException warnings involved in THRIFT-4805 +log4j.appender.console.filter.1=org.apache.log4j.varia.StringMatchFilter +log4j.appender.console.filter.1.StringToMatch=Thrift error occurred during processing of message +log4j.appender.console.filter.1.AcceptOnMatch=false diff --git a/core/pom.xml b/core/pom.xml index 4fb351ed1a11a..dd4fa242f3064 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js index 91bf274aa467d..8b32fe7d3e20d 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/stagepage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/stagepage.js @@ -43,6 +43,23 @@ $.extend( $.fn.dataTable.ext.type.order, { a = ConvertDurationString( a ); b = ConvertDurationString( b ); return ((a < b) ? 1 : ((a > b) ? -1 : 0)); + }, + + "size-pre": function (data) { + var floatValue = parseFloat(data) + return isNaN(floatValue) ? 0 : floatValue; + }, + + "size-asc": function (a, b) { + a = parseFloat(a); + b = parseFloat(b); + return ((a < b) ? -1 : ((a > b) ? 1 : 0)); + }, + + "size-desc": function (a, b) { + a = parseFloat(a); + b = parseFloat(b); + return ((a < b) ? 1 : ((a > b) ? -1 : 0)); } } ); @@ -562,10 +579,27 @@ $(document).ready(function () { } ], "columnDefs": [ - { "visible": false, "targets": 15 }, - { "visible": false, "targets": 16 }, - { "visible": false, "targets": 17 }, - { "visible": false, "targets": 18 } + // SPARK-35087 [type:size] means String with structures like : 'size / records', + // they should be sorted as numerical-order instead of lexicographical-order by default. + // The targets: $id represents column id which comes from stagespage-template.html + // #summary-executor-table.If the relative position of the columns in the table + // #summary-executor-table has changed,please be careful to adjust the column index here + // Input Size / Records + {"type": "size", "targets": 9}, + // Output Size / Records + {"type": "size", "targets": 10}, + // Shuffle Read Size / Records + {"type": "size", "targets": 11}, + // Shuffle Write Size / Records + {"type": "size", "targets": 12}, + // Peak JVM Memory OnHeap / OffHeap + {"visible": false, "targets": 15}, + // Peak Execution Memory OnHeap / OffHeap + {"visible": false, "targets": 16}, + // Peak Storage Memory OnHeap / OffHeap + {"visible": false, "targets": 17}, + // Peak Pool Memory Direct / Mapped + {"visible": false, "targets": 18} ], "deferRender": true, "order": [[0, "asc"]], @@ -746,7 +780,7 @@ $(document).ready(function () { "paging": true, "info": true, "processing": true, - "lengthMenu": [[20, 40, 60, 100, totalTasksToShow], [20, 40, 60, 100, "All"]], + "lengthMenu": [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], "orderMulti": false, "bAutoWidth": false, "ajax": { @@ -762,6 +796,9 @@ $(document).ready(function () { data.numTasks = totalTasksToShow; data.columnIndexToSort = columnIndexToSort; data.columnNameToSort = columnNameToSort; + if (data.length === -1) { + data.length = totalTasksToShow; + } }, "dataSrc": function (jsons) { var jsonStr = JSON.stringify(jsons); diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 262cee7b58aff..fad28ce333675 100755 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -113,7 +113,7 @@ table.sortable td { box-shadow: inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15); } -.progress.progress-started { +.progress .progress-bar.progress-started { background-color: #A0DFFF; background-image: -moz-linear-gradient(top, #A4EDFF, #94DDFF); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#A4EDFF), to(#94DDFF)); @@ -124,7 +124,7 @@ table.sortable td { filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } -.progress .progress-bar { +.progress .progress-bar.progress-completed { background-color: #3EC0FF; background-image: -moz-linear-gradient(top, #44CBFF, #34B0EE); background-image: -webkit-gradient(linear, 0 0, 0 100%, from(#44CBFF), to(#34B0EE)); diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index cfa1139140025..34b3089107efe 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -27,6 +27,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} +import org.apache.spark.scheduler.SparkListener import org.apache.spark.shuffle.api.ShuffleDriverComponents import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} @@ -39,6 +40,7 @@ private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask private case class CleanAccum(accId: Long) extends CleanupTask private case class CleanCheckpoint(rddId: Int) extends CleanupTask +private case class CleanSparkListener(listener: SparkListener) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -175,6 +177,13 @@ private[spark] class ContextCleaner( referenceBuffer.add(new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue)) } + /** Register a SparkListener to be cleaned up when its owner is garbage collected. */ + def registerSparkListenerForCleanup( + listenerOwner: AnyRef, + listener: SparkListener): Unit = { + registerForCleanup(listenerOwner, CleanSparkListener(listener)) + } + /** Keep cleaning RDD, shuffle, and broadcast state. */ private def keepCleaning(): Unit = Utils.tryOrStopSparkContext(sc) { while (!stopped) { @@ -197,6 +206,8 @@ private[spark] class ContextCleaner( doCleanupAccum(accId, blocking = blockOnCleanupTasks) case CleanCheckpoint(rddId) => doCleanCheckpoint(rddId) + case CleanSparkListener(listener) => + doCleanSparkListener(listener) } } } @@ -276,6 +287,16 @@ private[spark] class ContextCleaner( } } + def doCleanSparkListener(listener: SparkListener): Unit = { + try { + logDebug(s"Cleaning Spark listener $listener") + sc.listenerBus.removeListener(listener) + logDebug(s"Cleaned Spark listener $listener") + } catch { + case e: Exception => logError(s"Error cleaning Spark listener $listener", e) + } + } + private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index cdec1982b4487..ce71c2c7bc306 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import java.io.{ByteArrayInputStream, IOException, ObjectInputStream, ObjectOutputStream} import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.locks.ReentrantReadWriteLock @@ -100,7 +100,7 @@ private class ShuffleStatus(numPartitions: Int) extends Logging { * broadcast variable in order to keep it from being garbage collected and to allow for it to be * explicitly destroyed later on when the ShuffleMapStage is garbage-collected. */ - private[this] var cachedSerializedBroadcast: Broadcast[Array[Byte]] = _ + private[spark] var cachedSerializedBroadcast: Broadcast[Array[Byte]] = _ /** * Counter tracking the number of partitions that have output. This is a performance optimization @@ -843,7 +843,14 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr if (fetchedStatuses == null) { logInfo("Doing the fetch; tracker endpoint = " + trackerEndpoint) val fetchedBytes = askTracker[Array[Byte]](GetMapOutputStatuses(shuffleId)) - fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + try { + fetchedStatuses = MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + } catch { + case e: SparkException => + throw new MetadataFetchFailedException(shuffleId, -1, + s"Unable to deserialize broadcasted map statuses for shuffle $shuffleId: " + + e.getCause) + } logInfo("Got the output locations") mapStatuses.put(shuffleId, fetchedStatuses) } @@ -953,13 +960,19 @@ private[spark] object MapOutputTracker extends Logging { case DIRECT => deserializeObject(bytes, 1, bytes.length - 1).asInstanceOf[Array[MapStatus]] case BROADCAST => - // deserialize the Broadcast, pull .value array out of it, and then deserialize that - val bcast = deserializeObject(bytes, 1, bytes.length - 1). - asInstanceOf[Broadcast[Array[Byte]]] - logInfo("Broadcast mapstatuses size = " + bytes.length + - ", actual size = " + bcast.value.length) - // Important - ignore the DIRECT tag ! Start from offset 1 - deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]] + try { + // deserialize the Broadcast, pull .value array out of it, and then deserialize that + val bcast = deserializeObject(bytes, 1, bytes.length - 1). + asInstanceOf[Broadcast[Array[Byte]]] + logInfo("Broadcast mapstatuses size = " + bytes.length + + ", actual size = " + bcast.value.length) + // Important - ignore the DIRECT tag ! Start from offset 1 + deserializeObject(bcast.value, 1, bcast.value.length - 1).asInstanceOf[Array[MapStatus]] + } catch { + case e: IOException => + logWarning("Exception encountered during deserializing broadcasted map statuses: ", e) + throw new SparkException("Unable to deserialize broadcasted map statuses", e) + } case _ => throw new IllegalArgumentException("Unexpected byte tag = " + bytes(0)) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 17ceb5f1887c6..f0e8d335b7ebb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1583,11 +1583,7 @@ class SparkContext(config: SparkConf) extends Logging { private def addFile( path: String, recursive: Boolean, addedOnSubmit: Boolean, isArchive: Boolean = false ): Unit = { - val uri = if (!isArchive) { - new Path(path).toUri - } else { - Utils.resolveURI(path) - } + val uri = Utils.resolveURI(path) val schemeCorrectedURI = uri.getScheme match { case null => new File(path).getCanonicalFile.toURI case "local" => @@ -1619,10 +1615,8 @@ class SparkContext(config: SparkConf) extends Logging { env.rpcEnv.fileServer.addFile(new File(uri.getPath)) } else if (uri.getScheme == null) { schemeCorrectedURI.toString - } else if (isArchive) { - uri.toString } else { - path + uri.toString } val timestamp = if (addedOnSubmit) startTime else System.currentTimeMillis @@ -1977,7 +1971,7 @@ class SparkContext(config: SparkConf) extends Logging { // For local paths with backslashes on Windows, URI throws an exception addLocalJarFile(new File(path)) } else { - val uri = new Path(path).toUri + val uri = Utils.resolveURI(path) // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies Utils.validateURL(uri) uri.getScheme match { diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index dc2587a62ae40..dd962ca11ecc2 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -78,7 +78,8 @@ private[spark] object SerDeUtil extends Logging { * Choose batch size based on size of objects */ private[spark] class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { - private val pickle = new Pickler() + private val pickle = new Pickler(/* useMemo = */ true, + /* valueCompare = */ false) private var batch = 1 private val buffer = new mutable.ArrayBuffer[Any] @@ -131,7 +132,8 @@ private[spark] object SerDeUtil extends Logging { } private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { - val pickle = new Pickler + val pickle = new Pickler(/* useMemo = */ true, + /* valueCompare = */ false) val kt = Try { pickle.dumps(t._1) } @@ -182,7 +184,8 @@ private[spark] object SerDeUtil extends Logging { if (batchSize == 0) { new AutoBatchedPickler(cleaned) } else { - val pickle = new Pickler + val pickle = new Pickler(/* useMemo = */ true, + /* valueCompare = */ false) cleaned.grouped(batchSize).map(batched => pickle.dumps(batched.asJava)) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index bb3a20dce2da4..fa86da9ae9669 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -952,6 +952,15 @@ private[spark] class SparkSubmit extends Logging { } catch { case t: Throwable => throw findCause(t) + } finally { + if (!isShell(args.primaryResource) && !isSqlShell(args.mainClass) && + !isThriftServer(args.mainClass)) { + try { + SparkContext.getActive.foreach(_.stop()) + } catch { + case e: Throwable => logError(s"Failed to close SparkContext: $e") + } + } } } @@ -1186,7 +1195,7 @@ private[spark] object SparkSubmitUtils { sp.setM2compatible(true) sp.setUsepoms(true) sp.setRoot(sys.env.getOrElse( - "DEFAULT_ARTIFACT_REPOSITORY", "https://dl.bintray.com/spark-packages/maven")) + "DEFAULT_ARTIFACT_REPOSITORY", "https://repos.spark-packages.org/")) sp.setName("spark-packages") cr.add(sp) cr diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 9e1f753b51e5a..1dda6831b2214 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -309,7 +309,9 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {UIUtils.formatDuration(app.duration)} + + {UIUtils.formatDuration(app.duration)} + } 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 e7f1b8f3cf17a..4ead4397e9739 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -995,7 +995,7 @@ private[spark] class Executor( try { val response = heartbeatReceiverRef.askSync[HeartbeatResponse]( message, new RpcTimeout(HEARTBEAT_INTERVAL_MS.millis, EXECUTOR_HEARTBEAT_INTERVAL.key)) - if (response.reregisterBlockManager) { + if (!executorShutdown.get && response.reregisterBlockManager) { logInfo("Told to re-register on heartbeat") env.blockManager.reregister() } diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala index 80ef757332e43..5682a21e9560d 100644 --- a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -101,7 +101,8 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L } } - private def computeProcessTree(): Set[Int] = { + // Exposed for testing + private[executor] def computeProcessTree(): Set[Int] = { if (!isAvailable || testing) { return Set() } @@ -159,7 +160,8 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L } } - def addProcfsMetricsFromOneProcess( + // Exposed for testing + private[executor] def addProcfsMetricsFromOneProcess( allMetrics: ProcfsMetrics, pid: Int): ProcfsMetrics = { @@ -199,7 +201,7 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L case f: IOException => logWarning("There was a problem with reading" + " the stat file of the process. ", f) - ProcfsMetrics(0, 0, 0, 0, 0, 0) + throw f } } @@ -210,11 +212,16 @@ private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends L val pids = computeProcessTree var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0) for (p <- pids) { - allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) - // if we had an error getting any of the metrics, we don't want to report partial metrics, as - // that would be misleading. - if (!isAvailable) { - return ProcfsMetrics(0, 0, 0, 0, 0, 0) + try { + allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) + // if we had an error getting any of the metrics, we don't want to + // report partial metrics, as that would be misleading. + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } catch { + case _: IOException => + return ProcfsMetrics(0, 0, 0, 0, 0, 0) } } allMetrics 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 f6de5e4128ca5..3daa9f5362d9d 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 @@ -1015,7 +1015,7 @@ package object config { "like YARN and event logs.") .version("2.1.2") .regexConf - .createWithDefault("(?i)secret|password|token".r) + .createWithDefault("(?i)secret|password|token|access[.]key".r) private[spark] val STRING_REDACTION_PATTERN = ConfigBuilder("spark.redaction.string.regex") diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 0a9bb279d5210..7ea0f01511f27 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -200,13 +200,18 @@ class HadoopMapReduceCommitProtocol( val filesToMove = allAbsPathFiles.foldLeft(Map[String, String]())(_ ++ _) logDebug(s"Committing files staged for absolute locations $filesToMove") + val absParentPaths = filesToMove.values.map(new Path(_).getParent).toSet if (dynamicPartitionOverwrite) { - val absPartitionPaths = filesToMove.values.map(new Path(_).getParent).toSet - logDebug(s"Clean up absolute partition directories for overwriting: $absPartitionPaths") - absPartitionPaths.foreach(fs.delete(_, true)) + logDebug(s"Clean up absolute partition directories for overwriting: $absParentPaths") + absParentPaths.foreach(fs.delete(_, true)) } + logDebug(s"Create absolute parent directories: $absParentPaths") + absParentPaths.foreach(fs.mkdirs) for ((src, dst) <- filesToMove) { - fs.rename(new Path(src), new Path(dst)) + if (!fs.rename(new Path(src), new Path(dst))) { + throw new IOException(s"Failed to rename $src to $dst when committing files staged for " + + s"absolute locations") + } } if (dynamicPartitionOverwrite) { @@ -225,7 +230,11 @@ class HadoopMapReduceCommitProtocol( // a parent that exists, otherwise we may get unexpected result on the rename. fs.mkdirs(finalPartPath.getParent) } - fs.rename(new Path(stagingDir, part), finalPartPath) + val stagingPartPath = new Path(stagingDir, part) + if (!fs.rename(stagingPartPath, finalPartPath)) { + throw new IOException(s"Failed to rename $stagingPartPath to $finalPartPath when " + + s"committing files staged for overwriting dynamic partitions") + } } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala index 0f8fbd3ba2e9e..7cc2665ee7eee 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/PrometheusServlet.scala @@ -24,18 +24,15 @@ import com.codahale.metrics.MetricRegistry import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.annotation.Experimental import org.apache.spark.ui.JettyUtils._ /** - * :: Experimental :: * This exposes the metrics of the given registry with Prometheus format. * * The output is consistent with /metrics/json result in terms of item ordering * and with the previous result of Spark JMX Sink + Prometheus JMX Converter combination * in terms of key string format. */ -@Experimental private[spark] class PrometheusServlet( val property: Properties, val registry: MetricRegistry, diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f333ceee9f233..549f62788d110 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -890,10 +890,11 @@ private[spark] class DAGScheduler( timeout: Long, properties: Properties): PartialResult[R] = { val jobId = nextJobId.getAndIncrement() + val clonedProperties = Utils.cloneProperties(properties) if (rdd.partitions.isEmpty) { // Return immediately if the job is running 0 tasks val time = clock.getTimeMillis() - listenerBus.post(SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties)) + listenerBus.post(SparkListenerJobStart(jobId, time, Seq[StageInfo](), clonedProperties)) listenerBus.post(SparkListenerJobEnd(jobId, time, JobSucceeded)) return new PartialResult(evaluator.currentResult(), true) } @@ -901,7 +902,7 @@ private[spark] class DAGScheduler( val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, rdd.partitions.indices.toArray, callSite, listener, - Utils.cloneProperties(properties))) + clonedProperties)) listener.awaitResult() // Will throw an exception if the job fails } @@ -1162,7 +1163,8 @@ private[spark] class DAGScheduler( val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) listenerBus.post( - SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, + Utils.cloneProperties(properties))) submitStage(finalStage) } @@ -1200,7 +1202,8 @@ private[spark] class DAGScheduler( val stageIds = jobIdToStageIds(jobId).toArray val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) listenerBus.post( - SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, properties)) + SparkListenerJobStart(job.jobId, jobSubmissionTime, stageInfos, + Utils.cloneProperties(properties))) submitStage(finalStage) // If the whole stage has already finished, tell the listener and remove it @@ -1333,7 +1336,8 @@ private[spark] class DAGScheduler( } catch { case NonFatal(e) => stage.makeNewStageAttempt(partitionsToCompute.size) - listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, + Utils.cloneProperties(properties))) abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return @@ -1347,7 +1351,8 @@ private[spark] class DAGScheduler( if (partitionsToCompute.nonEmpty) { stage.latestInfo.submissionTime = Some(clock.getTimeMillis()) } - listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, + Utils.cloneProperties(properties))) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index bc2a0fbc36d5b..30c752960d5da 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -51,15 +51,17 @@ private[spark] class BlockStoreShuffleReader[K, C]( true } val useOldFetchProtocol = conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) + // SPARK-34790: Fetching continuous blocks in batch is incompatible with io encryption. + val ioEncryption = conf.get(config.IO_ENCRYPTION_ENABLED) val doBatchFetch = shouldBatchFetch && serializerRelocatable && - (!compressed || codecConcatenation) && !useOldFetchProtocol + (!compressed || codecConcatenation) && !useOldFetchProtocol && !ioEncryption if (shouldBatchFetch && !doBatchFetch) { logDebug("The feature tag of continuous shuffle block fetching is set to true, but " + "we can not enable the feature because other conditions are not satisfied. " + s"Shuffle compress: $compressed, serializer relocatable: $serializerRelocatable, " + s"codec concatenation: $codecConcatenation, use old shuffle fetch protocol: " + - s"$useOldFetchProtocol.") + s"$useOldFetchProtocol, io encryption: $ioEncryption.") } doBatchFetch } 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 52d41cdd72664..4245243c52be7 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -1018,7 +1018,7 @@ private[spark] class AppStatusListener( */ def activeStages(): Seq[v1.StageData] = { liveStages.values.asScala - .filter(_.info.submissionTime.isDefined) + .filter(s => Option(s.info).exists(_.submissionTime.isDefined)) .map(_.toApi()) .toList .sortBy(_.stageId) @@ -1179,7 +1179,7 @@ private[spark] class AppStatusListener( private def getOrCreateStage(info: StageInfo): LiveStage = { val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber), - (_: (Int, Int)) => new LiveStage()) + (_: (Int, Int)) => new LiveStage(info)) stage.info = info stage } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 38f1f25f2fcaa..d5cfdcb5842eb 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -393,14 +393,13 @@ private class LiveExecutorStageSummary( } -private class LiveStage extends LiveEntity { +private class LiveStage(var info: StageInfo) extends LiveEntity { import LiveEntityHelpers._ var jobs = Seq[LiveJob]() var jobIds = Set[Int]() - var info: StageInfo = null var status = v1.StageStatus.PENDING var description: Option[String] = None diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 5e3406037a72b..05b6fea38a9ab 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -460,13 +460,14 @@ private[spark] object UIUtils extends Logging { skipped: Int, reasonToNumKilled: Map[String, Int], total: Int): Seq[Node] = { - val ratio = if (total == 0) 100.0 else (completed.toDouble/total)*100 + val ratio = if (total == 0) 100.0 else (completed.toDouble / total) * 100 val completeWidth = "width: %s%%".format(ratio) // started + completed can be > total when there are speculative tasks val boundedStarted = math.min(started, total - completed) - val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100) + val startRatio = if (total == 0) 0.0 else (boundedStarted.toDouble / total) * 100 + val startWidth = "width: %s%%".format(startRatio) -
0) s"progress progress-started" else s"progress" }> +
{completed}/{total} { if (failed == 0 && skipped == 0 && started > 0) s"($started running)" } @@ -477,7 +478,8 @@ private[spark] object UIUtils extends Logging { } } -
+
+
} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index bb645f5958e96..1643aa68cdb5a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2065,6 +2065,17 @@ private[spark] object Utils extends Logging { } } + /** Check whether a path is an absolute URI. */ + def isAbsoluteURI(path: String): Boolean = { + try { + val uri = new URI(path: String) + uri.isAbsolute + } catch { + case _: URISyntaxException => + false + } + } + /** Return all non-local paths from a comma-separated list of paths. */ def nonLocalPaths(paths: String, testWindows: Boolean = false): Array[String] = { val windows = isWindows || testWindows @@ -2860,6 +2871,34 @@ private[spark] object Utils extends Logging { Hex.encodeHexString(secretBytes) } + /** + * Returns true if and only if the underlying class is a member class. + * + * Note: jdk8u throws a "Malformed class name" error if a given class is a deeply-nested + * inner class (See SPARK-34607 for details). This issue has already been fixed in jdk9+, so + * we can remove this helper method safely if we drop the support of jdk8u. + */ + def isMemberClass(cls: Class[_]): Boolean = { + try { + cls.isMemberClass + } catch { + case _: InternalError => + // We emulate jdk8u `Class.isMemberClass` below: + // public boolean isMemberClass() { + // return getSimpleBinaryName() != null && !isLocalOrAnonymousClass(); + // } + // `getSimpleBinaryName()` returns null if a given class is a top-level class, + // so we replace it with `cls.getEnclosingClass != null`. The second condition checks + // if a given class is not a local or an anonymous class, so we replace it with + // `cls.getEnclosingMethod == null` because `cls.getEnclosingMethod()` return a value + // only in either case (JVM Spec 4.8.6). + // + // Note: The newer jdk evaluates `!isLocalOrAnonymousClass()` first, + // we reorder the conditions to follow it. + cls.getEnclosingMethod == null && cls.getEnclosingClass != null + } + } + /** * Safer than Class obj's getSimpleName which may throw Malformed class name error in scala. * This method mimics scalatest's getSimpleNameOfAnObjectsClass. @@ -2972,6 +3011,9 @@ private[spark] object Utils extends Logging { /** Create a new properties object with the same values as `props` */ def cloneProperties(props: Properties): Properties = { + if (props == null) { + return props + } val resultProps = new Properties() props.forEach((k, v) => resultProps.put(k, v)) resultProps diff --git a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala index 1d3e28b39548f..8f220801f41e3 100644 --- a/core/src/test/scala/org/apache/spark/DebugFilesystem.scala +++ b/core/src/test/scala/org/apache/spark/DebugFilesystem.scala @@ -57,8 +57,14 @@ object DebugFilesystem extends Logging { } /** - * DebugFilesystem wraps file open calls to track all open connections. This can be used in tests - * to check that connections are not leaked. + * DebugFilesystem wraps + * 1) file open calls to track all open connections. This can be used in tests to check that + * connections are not leaked; + * 2) rename calls to return false when destination's parent path does not exist. When + * destination parent does not exist, LocalFileSystem uses FileUtil#copy to copy the + * file and returns true if succeed, while many other hadoop file systems (e.g. HDFS, S3A) + * return false without renaming any file. This helps to test that Spark can work with the + * latter file systems. */ // TODO(ekl) we should consider always interposing this to expose num open conns as a metric class DebugFilesystem extends LocalFileSystem { @@ -120,4 +126,8 @@ class DebugFilesystem extends LocalFileSystem { override def hashCode(): Int = wrapped.hashCode() } } + + override def rename(src: Path, dst: Path): Boolean = { + exists(dst.getParent) && super.rename(src, dst) + } } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index b5b68f639ffc9..33e1113774663 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -333,4 +333,45 @@ class MapOutputTrackerSuite extends SparkFunSuite { rpcEnv.shutdown() } + test("SPARK-34939: remote fetch using broadcast if broadcasted value is destroyed") { + val newConf = new SparkConf + newConf.set(RPC_MESSAGE_MAX_SIZE, 1) + newConf.set(RPC_ASK_TIMEOUT, "1") // Fail fast + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 10240L) // 10 KiB << 1MiB framesize + + // needs TorrentBroadcast so need a SparkContext + withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => + val masterTracker = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + val rpcEnv = sc.env.rpcEnv + val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) + rpcEnv.stop(masterTracker.trackerEndpoint) + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 5)) + } + + val mapWorkerRpcEnv = createRpcEnv("spark-worker", "localhost", 0, new SecurityManager(conf)) + val mapWorkerTracker = new MapOutputTrackerWorker(conf) + mapWorkerTracker.trackerEndpoint = + mapWorkerRpcEnv.setupEndpointRef(rpcEnv.address, MapOutputTracker.ENDPOINT_NAME) + + val fetchedBytes = mapWorkerTracker.trackerEndpoint + .askSync[Array[Byte]](GetMapOutputStatuses(20)) + assert(fetchedBytes(0) == 1) + + // Normally `unregisterMapOutput` triggers the destroy of broadcasted value. + // But the timing of destroying broadcasted value is indeterminate, we manually destroy + // it by blocking. + masterTracker.shuffleStatuses.get(20).foreach { shuffleStatus => + shuffleStatus.cachedSerializedBroadcast.destroy(true) + } + val err = intercept[SparkException] { + MapOutputTracker.deserializeMapStatuses(fetchedBytes, conf) + } + assert(err.getMessage.contains("Unable to deserialize broadcasted map statuses")) + } + } } diff --git a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala index 78f1246295bf8..e433f429000c1 100644 --- a/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/MapStatusesSerDeserBenchmark.scala @@ -17,8 +17,6 @@ package org.apache.spark -import org.scalatest.Assertions._ - import org.apache.spark.benchmark.Benchmark import org.apache.spark.benchmark.BenchmarkBase import org.apache.spark.scheduler.CompressedMapStatus @@ -28,7 +26,7 @@ import org.apache.spark.storage.BlockManagerId * Benchmark for MapStatuses serialization & deserialization performance. * {{{ * To run this benchmark: - * 1. without sbt: bin/spark-submit --class --jars + * 1. without sbt: bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " * Results will be written to "benchmarks/MapStatusesSerDeserBenchmark-results.txt". diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 0c0a9b86ac603..c4bcccfd1d45a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -1069,6 +1069,46 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.hadoopConfiguration.get(bufferKey).toInt === 65536, "spark configs have higher priority than spark.hadoop configs") } + + test("SPARK-34225: addFile/addJar shouldn't further encode URI if a URI form string is passed") { + withTempDir { dir => + val jar1 = File.createTempFile("testprefix", "test jar.jar", dir) + val jarUrl1 = jar1.toURI.toString + val file1 = File.createTempFile("testprefix", "test file.txt", dir) + val fileUrl1 = file1.toURI.toString + val jar2 = File.createTempFile("testprefix", "test %20jar.jar", dir) + val file2 = File.createTempFile("testprefix", "test %20file.txt", dir) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(jarUrl1) + sc.addFile(fileUrl1) + sc.addJar(jar2.toString) + sc.addFile(file2.toString) + sc.parallelize(Array(1), 1).map { x => + val gottenJar1 = new File(SparkFiles.get(jar1.getName)) + if (!gottenJar1.exists()) { + throw new SparkException("file doesn't exist : " + jar1) + } + val gottenFile1 = new File(SparkFiles.get(file1.getName)) + if (!gottenFile1.exists()) { + throw new SparkException("file doesn't exist : " + file1) + } + val gottenJar2 = new File(SparkFiles.get(jar2.getName)) + if (!gottenJar2.exists()) { + throw new SparkException("file doesn't exist : " + jar2) + } + val gottenFile2 = new File(SparkFiles.get(file2.getName)) + if (!gottenFile2.exists()) { + throw new SparkException("file doesn't exist : " + file2) + } + x + }.collect() + } finally { + sc.stop() + } + } + } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 9296274453c38..07ef46d048ea0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -418,12 +418,7 @@ class MasterSuite extends SparkFunSuite (workerResponse \ "masterwebuiurl").extract[String] should be (reverseProxyUrl + "/") } - // with LocalCluster, we have masters and workers in the same JVM, each overwriting - // system property spark.ui.proxyBase. - // so we need to manage this property explicitly for test - System.getProperty("spark.ui.proxyBase") should startWith - (s"$reverseProxyUrl/proxy/worker-") - System.setProperty("spark.ui.proxyBase", reverseProxyUrl) + System.getProperty("spark.ui.proxyBase") should be (reverseProxyUrl) val html = Utils .tryWithResource(Source.fromURL(s"$masterUrl/"))(_.getLines().mkString("\n")) html should include ("Spark Master at spark://") diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 97ffb36062dbc..a237447b0fa2d 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -270,6 +270,17 @@ class ExecutorSuite extends SparkFunSuite heartbeatZeroAccumulatorUpdateTest(false) } + private def withMockHeartbeatReceiverRef(executor: Executor) + (func: RpcEndpointRef => Unit): Unit = { + val executorClass = classOf[Executor] + val mockReceiverRef = mock[RpcEndpointRef] + val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") + receiverRef.setAccessible(true) + receiverRef.set(executor, mockReceiverRef) + + func(mockReceiverRef) + } + private def withHeartbeatExecutor(confs: (String, String)*) (f: (Executor, ArrayBuffer[Heartbeat]) => Unit): Unit = { val conf = new SparkConf @@ -277,22 +288,18 @@ class ExecutorSuite extends SparkFunSuite val serializer = new JavaSerializer(conf) val env = createMockEnv(conf, serializer) withExecutor("id", "localhost", SparkEnv.get) { executor => - val executorClass = classOf[Executor] - - // Save all heartbeats sent into an ArrayBuffer for verification - val heartbeats = ArrayBuffer[Heartbeat]() - val mockReceiver = mock[RpcEndpointRef] - when(mockReceiver.askSync(any[Heartbeat], any[RpcTimeout])(any)) - .thenAnswer((invocation: InvocationOnMock) => { - val args = invocation.getArguments() - heartbeats += args(0).asInstanceOf[Heartbeat] - HeartbeatResponse(false) - }) - val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") - receiverRef.setAccessible(true) - receiverRef.set(executor, mockReceiver) + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + // Save all heartbeats sent into an ArrayBuffer for verification + val heartbeats = ArrayBuffer[Heartbeat]() + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)) + .thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments() + heartbeats += args(0).asInstanceOf[Heartbeat] + HeartbeatResponse(false) + }) - f(executor, heartbeats) + f(executor, heartbeats) + } } } @@ -416,6 +423,35 @@ class ExecutorSuite extends SparkFunSuite assert(taskMetrics.getMetricValue("JVMHeapMemory") > 0) } + test("SPARK-34949: do not re-register BlockManager when executor is shutting down") { + val reregisterInvoked = new AtomicBoolean(false) + val mockBlockManager = mock[BlockManager] + when(mockBlockManager.reregister()).thenAnswer { (_: InvocationOnMock) => + reregisterInvoked.getAndSet(true) + } + val conf = new SparkConf(false).setAppName("test").setMaster("local[2]") + val mockEnv = createMockEnv(conf, new JavaSerializer(conf)) + when(mockEnv.blockManager).thenReturn(mockBlockManager) + + withExecutor("id", "localhost", mockEnv) { executor => + withMockHeartbeatReceiverRef(executor) { mockReceiverRef => + when(mockReceiverRef.askSync(any[Heartbeat], any[RpcTimeout])(any)).thenAnswer { + (_: InvocationOnMock) => HeartbeatResponse(reregisterBlockManager = true) + } + val reportHeartbeat = PrivateMethod[Unit](Symbol("reportHeartBeat")) + executor.invokePrivate(reportHeartbeat()) + assert(reregisterInvoked.get(), "BlockManager.reregister should be invoked " + + "on HeartbeatResponse(reregisterBlockManager = true) when executor is not shutting down") + + reregisterInvoked.getAndSet(false) + executor.stop() + executor.invokePrivate(reportHeartbeat()) + assert(!reregisterInvoked.get(), + "BlockManager.reregister should not be invoked when executor is shutting down") + } + } + } + test("SPARK-33587: isFatalError") { def errorInThreadPool(e: => Throwable): Throwable = { intercept[Throwable] { diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala index 9836697e1647c..ff0374da1bcfe 100644 --- a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import org.mockito.Mockito.{spy, when} + import org.apache.spark.SparkFunSuite @@ -38,4 +40,26 @@ class ProcfsMetricsGetterSuite extends SparkFunSuite { assert(r.jvmVmemTotal == 4769947648L) assert(r.jvmRSSTotal == 262610944) } + + test("SPARK-34845: partial metrics shouldn't be returned") { + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + val mockedP = spy(p) + + var ptree: Set[Int] = Set(26109, 22763) + when(mockedP.computeProcessTree).thenReturn(ptree) + var r = mockedP.computeAllMetrics + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + assert(r.pythonVmemTotal == 360595456) + assert(r.pythonRSSTotal == 7831552) + + // proc file of pid 22764 doesn't exist, so partial metrics shouldn't be returned + ptree = Set(26109, 22764, 22763) + when(mockedP.computeProcessTree).thenReturn(ptree) + r = mockedP.computeAllMetrics + assert(r.jvmVmemTotal == 0) + assert(r.jvmRSSTotal == 0) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala b/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala index 617ca5a1a8bc4..f62c561ec550c 100644 --- a/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala @@ -29,7 +29,7 @@ import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index 525e682dd5d42..1c17d7b1392ab 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -31,7 +31,7 @@ import org.apache.spark.serializer.KryoTest._ * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala index dde0c98704d00..7f5dbff46b776 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.ThreadUtils * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala index d4368c882d660..9d040bb4e1ec7 100644 --- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala @@ -113,7 +113,8 @@ class UIUtilsSuite extends SparkFunSuite { test("SPARK-11906: Progress bar should not overflow because of speculative tasks") { val generated = makeProgressBar(2, 3, 0, 0, Map.empty, 4).head.child.filter(_.label == "div") val expected = Seq( -
+
, +
) assert(generated.sameElements(expected), s"\nRunning progress bar should round down\n\nExpected:\n$expected\nGenerated:\n$generated") diff --git a/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala index baacc7527a806..ff4a4941b6b9e 100644 --- a/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/util/PropertiesCloneBenchmark.scala @@ -26,11 +26,11 @@ import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} /** - * Benchmark for Kryo Unsafe vs safe Serialization. + * Benchmark for SerializationUtils.clone vs Utils.cloneProperties. * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 18ff96021153f..208e7297f06aa 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1024,11 +1024,13 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { // Set some secret keys val secretKeys = Seq( "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD", + "spark.hadoop.fs.s3a.access.key", "spark.my.password", "spark.my.sECreT") secretKeys.foreach { key => sparkConf.set(key, "sensitive_value") } // Set a non-secret key sparkConf.set("spark.regular.property", "regular_value") + sparkConf.set("spark.hadoop.fs.s3a.access_key", "regular_value") // Set a property with a regular key but secret in the value sparkConf.set("spark.sensitive.property", "has_secret_in_value") @@ -1039,7 +1041,8 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { secretKeys.foreach { key => assert(redactedConf(key) === Utils.REDACTION_REPLACEMENT_TEXT) } assert(redactedConf("spark.regular.property") === "regular_value") assert(redactedConf("spark.sensitive.property") === Utils.REDACTION_REPLACEMENT_TEXT) - + assert(redactedConf("spark.hadoop.fs.s3a.access.key") === Utils.REDACTION_REPLACEMENT_TEXT) + assert(redactedConf("spark.hadoop.fs.s3a.access_key") === "regular_value") } test("redact sensitive information in command line args") { diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomBenchmark.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomBenchmark.scala index b0563fdb2badb..7fd63a2c3518f 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomBenchmark.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.Utils.times * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class * 2. build/sbt "core/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " diff --git a/dev/change-scala-version.sh b/dev/change-scala-version.sh index 06411b9b12a0d..6adfb237f18b3 100755 --- a/dev/change-scala-version.sh +++ b/dev/change-scala-version.sh @@ -60,6 +60,17 @@ BASEDIR=$(dirname $0)/.. find "$BASEDIR" -name 'pom.xml' -not -path '*target*' -print \ -exec bash -c "sed_i 's/\(artifactId.*\)_'$FROM_VERSION'/\1_'$TO_VERSION'/g' {}" \; +# dependency:get is workaround for SPARK-34762 to download the JAR file of commons-cli. +# Without this, build with Scala 2.13 using SBT will fail because the help plugin used below downloads only the POM file. +COMMONS_CLI_VERSION=`build/mvn help:evaluate -Dexpression=commons-cli.version -q -DforceStdout` +build/mvn dependency:get -Dartifact=commons-cli:commons-cli:${COMMONS_CLI_VERSION} -q + +# Update in parent POM +# First find the right full version from the profile's build +SCALA_VERSION=`build/mvn help:evaluate -Pscala-${TO_VERSION} -Dexpression=scala.version -q -DforceStdout` +sed_i '1,/[0-9]*\.[0-9]*\.[0-9]*[0-9]*\.[0-9]*\.[0-9]*'$SCALA_VERSION' in parent POM # Match any scala binary version to ensure idempotency sed_i '1,/[0-9]*\.[0-9]*[0-9]*\.[0-9]*'$TO_VERSION' last_index: - response = raw_input(" Please enter an integer between 0 and %d: " % last_index) + response = input(" Please enter an integer between 0 and %d: " % last_index) response = int(response) if response == custom_index: - new_author = raw_input(" Please type a custom name for this author: ") + new_author = input(" Please type a custom name for this author: ") elif response != raw_index: new_author = candidate_names[response] # In non-interactive mode, just pick the first candidate diff --git a/dev/requirements.txt b/dev/requirements.txt index c1546c8b8d4d3..ddb3e1729f03f 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -1,7 +1,6 @@ flake8==3.5.0 jira==1.0.3 PyGithub==1.26.0 -Unidecode==0.04.19 sphinx pydata_sphinx_theme ipython diff --git a/dev/run-pip-tests b/dev/run-pip-tests index b322d3f61b444..cb64e88318a89 100755 --- a/dev/run-pip-tests +++ b/dev/run-pip-tests @@ -80,10 +80,6 @@ for python in "${PYTHON_EXECS[@]}"; do VIRTUALENV_PATH="$VIRTUALENV_BASE"/$python rm -rf "$VIRTUALENV_PATH" if [ -n "$USE_CONDA" ]; then - if [ -f "$CONDA_PREFIX/etc/profile.d/conda.sh" ]; then - # See also https://github.com/conda/conda/issues/7980 - source "$CONDA_PREFIX/etc/profile.d/conda.sh" - fi conda create -y -p "$VIRTUALENV_PATH" python=$python numpy pandas pip setuptools source activate "$VIRTUALENV_PATH" || conda activate "$VIRTUALENV_PATH" else diff --git a/docs/README.md b/docs/README.md index dd3ee86efaa12..27494f723ff27 100644 --- a/docs/README.md +++ b/docs/README.md @@ -66,10 +66,13 @@ To generate API docs for any language, you'll need to install these libraries: ```sh -$ sudo pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ipython nbsphinx numpydoc +$ sudo pip install 'sphinx<3.1.0' mkdocs numpy pydata_sphinx_theme ipython nbsphinx numpydoc 'jinja2<3.0.0' ``` ## Generating the Documentation HTML diff --git a/docs/_config.yml b/docs/_config.yml index 21af9df87b571..6a7f4759d46aa 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -19,8 +19,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 3.1.1 -SPARK_VERSION_SHORT: 3.1.1 +SPARK_VERSION: 3.1.2 +SPARK_VERSION_SHORT: 3.1.2 SCALA_BINARY_VERSION: "2.12" SCALA_VERSION: "2.12.10" MESOS_VERSION: 1.0.0 diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index cda2a1a5139a1..1123521087be8 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -188,6 +188,8 @@ url: sql-ref-syntax-qry-select-lateral-view.html - text: PIVOT Clause url: sql-ref-syntax-qry-select-pivot.html + - text: TRANSFORM Clause + url: sql-ref-syntax-qry-select-transform.html - text: EXPLAIN url: sql-ref-syntax-qry-explain.html - text: Auxiliary Statements diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 1eaa8ab4eeee0..8994b0aac99b7 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -250,7 +250,8 @@ More details on these committers can be found in the latest Hadoop documentation Here is the documentation on the standard connectors both from Apache and the cloud providers. * [OpenStack Swift](https://hadoop.apache.org/docs/current/hadoop-openstack/index.html). -* [Azure Blob Storage and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-aws/tools/hadoop-aws/index.html). +* [Azure Blob Storage](https://hadoop.apache.org/docs/current/hadoop-azure/index.html). +* [Azure Blob Filesystem (ABFS) and Azure Datalake Gen 2](https://hadoop.apache.org/docs/current/hadoop-azure/abfs.html). * [Azure Data Lake Gen 1](https://hadoop.apache.org/docs/current/hadoop-azure-datalake/index.html). * [Hadoop-AWS module (Hadoop 3.x)](https://hadoop.apache.org/docs/current3/hadoop-aws/tools/hadoop-aws/index.html). * [Amazon S3 via S3A and S3N (Hadoop 2.x)](https://hadoop.apache.org/docs/current2/hadoop-aws/tools/hadoop-aws/index.html). diff --git a/docs/ml-features.md b/docs/ml-features.md index b36b0761333eb..e01acfd0b979d 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -1497,8 +1497,8 @@ for more details on the API. ## Imputer -The `Imputer` estimator completes missing values in a dataset, either using the mean or the -median of the columns in which the missing values are located. The input columns should be of +The `Imputer` estimator completes missing values in a dataset, using the mean, median or mode +of the columns in which the missing values are located. The input columns should be of numeric type. Currently `Imputer` does not support categorical features and possibly creates incorrect values for columns containing categorical features. Imputer can impute custom values other than 'NaN' by `.setMissingValue(custom_value)`. For example, `.setMissingValue(0)` will impute diff --git a/docs/monitoring.md b/docs/monitoring.md index 5b3278bca031d..930f91f9a5c2f 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -755,7 +755,7 @@ A list of the available metrics, with a short description: Executor-level metrics are sent from each executor to the driver as part of the Heartbeat to describe the performance metrics of Executor itself like JVM heap memory, GC information. Executor metric values and their measured memory peak values per executor are exposed via the REST API in JSON format and in Prometheus format. The JSON end point is exposed at: `/applications/[app-id]/executors`, and the Prometheus endpoint at: `/metrics/executors/prometheus`. -The Prometheus endpoint is experimental and conditional to a configuration parameter: `spark.ui.prometheus.enabled=true` (the default is `false`). +The Prometheus endpoint is conditional to a configuration parameter: `spark.ui.prometheus.enabled=true` (the default is `false`). In addition, aggregated per-stage peak values of the executor memory metrics are written to the event log if `spark.eventLog.logStageExecutorMetrics` is true. Executor memory metrics are also exposed via the Spark metrics system based on the [Dropwizard metrics library](http://metrics.dropwizard.io/4.1.1). diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index cf5c01629a240..b9a018acf1e26 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -212,7 +212,7 @@ A typical example of this using S3 is via passing the following options: ``` ... ---packages com.amazonaws:aws-java-sdk:1.7.4,org.apache.hadoop:hadoop-aws:2.7.6 +--packages org.apache.hadoop:hadoop-aws:3.2.0 --conf spark.kubernetes.file.upload.path=s3a:///path --conf spark.hadoop.fs.s3a.access.key=... --conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 7d60915e2a65e..89a025ce6560f 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -211,6 +211,25 @@ the following case-insensitive options: Specifies kerberos principal name for the JDBC client. If both keytab and principal are defined then Spark tries to do kerberos authentication. + + + refreshKrb5Config + + This option controls whether the kerberos configuration is to be refreshed or not for the JDBC client before + establishing a new connection. Set to true if you want to refresh the configuration, otherwise set to false. + The default value is false. Note that if you set this option to true and try to establish multiple connections, + a race condition can occur. One possble situation would be like as follows. +
    +
  1. refreshKrb5Config flag is set with security context 1
  2. +
  3. A JDBC connection provider is used for the corresponding DBMS
  4. +
  5. The krb5.conf is modified but the JVM not yet realized that it must be reloaded
  6. +
  7. Spark authenticates successfully for security context 1
  8. +
  9. The JVM loads security context 2 from the modified krb5.conf
  10. +
  11. Spark restores the previously saved security context 1
  12. +
  13. The modified krb5.conf content just gone
  14. +
+ +
diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 440d380ac5cdc..75dafe400f3f4 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -111,7 +111,7 @@ license: | - In Spark 3.0, `SHOW TBLPROPERTIES` throws `AnalysisException` if the table does not exist. In Spark version 2.4 and below, this scenario caused `NoSuchTableException`. - - In Spark 3.0, `SHOW CREATE TABLE` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE AS SERDE` command instead. + - In Spark 3.0, `SHOW CREATE TABLE table_identifier` always returns Spark DDL, even when the given table is a Hive SerDe table. For generating Hive DDL, use `SHOW CREATE TABLE table_identifier AS SERDE` command instead. - In Spark 3.0, column of CHAR type is not allowed in non-Hive-Serde tables, and CREATE/ALTER TABLE commands will fail if CHAR type is detected. Please use STRING type instead. In Spark version 2.4 and below, CHAR type is treated as STRING type and the length parameter is simply ignored. @@ -874,7 +874,7 @@ Spark SQL supports the vast majority of Hive features, such as: * All Hive operators, including: * Relational operators (`=`, `<=>`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Logical operators (`AND`, `OR`, etc) * Complex type constructors * Mathematical functions (`sign`, `ln`, `cos`, etc) * String functions (`instr`, `length`, `printf`, etc) diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md index e99af41635c9d..bdfe6cde8a728 100644 --- a/docs/sql-performance-tuning.md +++ b/docs/sql-performance-tuning.md @@ -255,9 +255,9 @@ This feature coalesces the post shuffle partitions based on the map output stati spark.sql.adaptive.coalescePartitions.initialPartitionNum - 200 + (none) - The initial number of shuffle partitions before coalescing. By default it equals to spark.sql.shuffle.partitions. This configuration only has an effect when spark.sql.adaptive.enabled and spark.sql.adaptive.coalescePartitions.enabled are both enabled. + The initial number of shuffle partitions before coalescing. If not set, it equals to spark.sql.shuffle.partitions. This configuration only has an effect when spark.sql.adaptive.enabled and spark.sql.adaptive.coalescePartitions.enabled are both enabled. 3.0.0 @@ -288,7 +288,7 @@ Data skew can severely downgrade the performance of join queries. This feature d spark.sql.adaptive.skewJoin.skewedPartitionFactor - 10 + 5 A partition is considered as skewed if its size is larger than this factor multiplying the median partition size and also larger than spark.sql.adaptive.skewJoin.skewedPartitionThresholdInBytes. diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md index 9b9a7df7f612f..5300232fe4ee5 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-file.md @@ -26,7 +26,7 @@ license: | ### Syntax ```sql -LIST FILE +LIST { FILE | FILES } file_name [ ... ] ``` ### Examples diff --git a/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md index 04aa52c2ad8af..cfe8def8ef330 100644 --- a/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md +++ b/docs/sql-ref-syntax-aux-resource-mgmt-list-jar.md @@ -26,7 +26,7 @@ license: | ### Syntax ```sql -LIST JAR +LIST { JAR | JARS } file_name [ ... ] ``` ### Examples diff --git a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md index 11ec2f1d9ea85..b2f5957416a80 100644 --- a/docs/sql-ref-syntax-ddl-create-table-hiveformat.md +++ b/docs/sql-ref-syntax-ddl-create-table-hiveformat.md @@ -39,14 +39,6 @@ CREATE [ EXTERNAL ] TABLE [ IF NOT EXISTS ] table_identifier [ LOCATION path ] [ TBLPROPERTIES ( key1=val1, key2=val2, ... ) ] [ AS select_statement ] - -row_format: - : SERDE serde_class [ WITH SERDEPROPERTIES (k1=v1, k2=v2, ... ) ] - | DELIMITED [ FIELDS TERMINATED BY fields_terminated_char [ ESCAPED BY escaped_char ] ] - [ COLLECTION ITEMS TERMINATED BY collection_items_terminated_char ] - [ MAP KEYS TERMINATED BY map_key_terminated_char ] - [ LINES TERMINATED BY row_terminated_char ] - [ NULL DEFINED AS null_char ] ``` Note that, the clauses between the columns definition clause and the AS SELECT clause can come in @@ -82,50 +74,10 @@ as any order. For example, you can write COMMENT table_comment after TBLPROPERTI * **INTO num_buckets BUCKETS** Specifies buckets numbers, which is used in `CLUSTERED BY` clause. - -* **row_format** - - Use the `SERDE` clause to specify a custom SerDe for one table. Otherwise, use the `DELIMITED` clause to use the native SerDe and specify the delimiter, escape character, null character and so on. - -* **SERDE** - - Specifies a custom SerDe for one table. - -* **serde_class** - - Specifies a fully-qualified class name of a custom SerDe. - -* **SERDEPROPERTIES** - - A list of key-value pairs that is used to tag the SerDe definition. - -* **DELIMITED** - The `DELIMITED` clause can be used to specify the native SerDe and state the delimiter, escape character, null character and so on. - -* **FIELDS TERMINATED BY** - - Used to define a column separator. - -* **COLLECTION ITEMS TERMINATED BY** - - Used to define a collection item separator. - -* **MAP KEYS TERMINATED BY** - - Used to define a map key separator. - -* **LINES TERMINATED BY** - - Used to define a row separator. - -* **NULL DEFINED AS** - - Used to define the specific value for NULL. - -* **ESCAPED BY** +* **row_format** - Used for escape mechanism. + Specifies the row format for input and output. See [HIVE FORMAT](sql-ref-syntax-hive-format.html) for more syntax details. * **STORED AS** diff --git a/docs/sql-ref-syntax-dml-insert-into.md b/docs/sql-ref-syntax-dml-insert-into.md index 15400780289e2..fc7b4250a71b6 100644 --- a/docs/sql-ref-syntax-dml-insert-into.md +++ b/docs/sql-ref-syntax-dml-insert-into.md @@ -70,8 +70,8 @@ INSERT INTO [ TABLE ] table_identifier [ partition_spec ] [ ( column_list ) ] #### Single Row Insert Using a VALUES Clause ```sql -CREATE TABLE students (name VARCHAR(64), address VARCHAR(64), student_id INT) - USING PARQUET PARTITIONED BY (student_id); +CREATE TABLE students (name VARCHAR(64), address VARCHAR(64)) + USING PARQUET PARTITIONED BY (student_id INT); INSERT INTO students VALUES ('Amy Smith', '123 Park Ave, San Jose', 111111); diff --git a/docs/sql-ref-syntax-hive-format.md b/docs/sql-ref-syntax-hive-format.md new file mode 100644 index 0000000000000..8092e582d97ad --- /dev/null +++ b/docs/sql-ref-syntax-hive-format.md @@ -0,0 +1,73 @@ +--- +layout: global +title: Hive Row Format +displayTitle: Hive Row Format +license: | + 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. +--- + +### Description + +Spark supports a Hive row format in `CREATE TABLE` and `TRANSFORM` clause to specify serde or text delimiter. +There are two ways to define a row format in `row_format` of `CREATE TABLE` and `TRANSFORM` clauses. + 1. `SERDE` clause to specify a custom SerDe class. + 2. `DELIMITED` clause to specify a delimiter, an escape character, a null character, and so on for the native SerDe. + +### Syntax + +```sql +row_format: + SERDE serde_class [ WITH SERDEPROPERTIES (k1=v1, k2=v2, ... ) ] + | DELIMITED [ FIELDS TERMINATED BY fields_terminated_char [ ESCAPED BY escaped_char ] ] + [ COLLECTION ITEMS TERMINATED BY collection_items_terminated_char ] + [ MAP KEYS TERMINATED BY map_key_terminated_char ] + [ LINES TERMINATED BY row_terminated_char ] + [ NULL DEFINED AS null_char ] +``` + +### Parameters + +* **SERDE serde_class** + + Specifies a fully-qualified class name of custom SerDe. + +* **SERDEPROPERTIES** + + A list of key-value pairs that is used to tag the SerDe definition. + +* **FIELDS TERMINATED BY** + + Used to define a column separator. + +* **COLLECTION ITEMS TERMINATED BY** + + Used to define a collection item separator. + +* **MAP KEYS TERMINATED BY** + + Used to define a map key separator. + +* **LINES TERMINATED BY** + + Used to define a row separator. + +* **NULL DEFINED AS** + + Used to define the specific value for NULL. + +* **ESCAPED BY** + + Used for escape mechanism. diff --git a/docs/sql-ref-syntax-qry-select-transform.md b/docs/sql-ref-syntax-qry-select-transform.md new file mode 100644 index 0000000000000..21966f2e1cc34 --- /dev/null +++ b/docs/sql-ref-syntax-qry-select-transform.md @@ -0,0 +1,191 @@ +--- +layout: global +title: TRANSFORM +displayTitle: TRANSFORM +license: | + 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. +--- + +### Description + +The `TRANSFORM` clause is used to specify a Hive-style transform query specification +to transform the inputs by running a user-specified command or script. + +### Syntax + +```sql +SELECT TRANSFORM ( expression [ , ... ] ) + [ ROW FORMAT row_format ] + [ RECORDWRITER record_writer_class ] + USING command_or_script [ AS ( [ col_name [ col_type ] ] [ , ... ] ) ] + [ ROW FORMAT row_format ] + [ RECORDREADER record_reader_class ] +``` + +### Parameters + +* **expression** + + Specifies a combination of one or more values, operators and SQL functions that results in a value. + +* **row_format** + + Specifies the row format for input and output. See [HIVE FORMAT](sql-ref-syntax-hive-format.html) for more syntax details. + +* **RECORDWRITER** + + Specifies a fully-qualified class name of a custom RecordWriter. The default value is `org.apache.hadoop.hive.ql.exec.TextRecordWriter`. + +* **RECORDREADER** + + Specifies a fully-qualified class name of a custom RecordReader. The default value is `org.apache.hadoop.hive.ql.exec.TextRecordReader`. + +* **command_or_script** + + Specifies a command or a path to script to process data. + +### SerDe behavior + +Spark uses the Hive SerDe `org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe` by default, so columns will be casted +to `STRING` and combined by tabs before feeding to the user script. All `NULL` values will be converted +to the literal string `"\N"` in order to differentiate `NULL` values from empty strings. The standard output of the +user script will be treated as tab-separated `STRING` columns, any cell containing only `"\N"` will be re-interpreted +as a `NULL` value, and then the resulting STRING column will be cast to the data type specified in `col_type`. If the actual +number of output columns is less than the number of specified output columns, insufficient output columns will be +supplemented with `NULL`. If the actual number of output columns is more than the number of specified output columns, +the output columns will only select the corresponding columns and the remaining part will be discarded. +If there is no `AS` clause after `USING my_script`, an output schema will be `key: STRING, value: STRING`. +The `key` column contains all the characters before the first tab and the `value` column contains the remaining characters after the first tab. +If there is no enough tab, Spark will return `NULL` value. These defaults can be overridden with `ROW FORMAT SERDE` or `ROW FORMAT DELIMITED`. + +### Examples + +```sql +CREATE TABLE person (zip_code INT, name STRING, age INT); +INSERT INTO person VALUES + (94588, 'Zen Hui', 50), + (94588, 'Dan Li', 18), + (94588, 'Anil K', 27), + (94588, 'John V', NULL), + (94511, 'David K', 42), + (94511, 'Aryan B.', 18), + (94511, 'Lalit B.', NULL); + +-- With specified output without data type +SELECT TRANSFORM(zip_code, name, age) + USING 'cat' AS (a, b, c) +FROM person +WHERE zip_code > 94511; ++-------+---------+-----+ +| a | b| c| ++-------+---------+-----+ +| 94588| Anil K| 27| +| 94588| John V| NULL| +| 94588| Zen Hui| 50| +| 94588| Dan Li| 18| ++-------+---------+-----+ + +-- With specified output with data type +SELECT TRANSFORM(zip_code, name, age) + USING 'cat' AS (a STRING, b STRING, c STRING) +FROM person +WHERE zip_code > 94511; ++-------+---------+-----+ +| a | b| c| ++-------+---------+-----+ +| 94588| Anil K| 27| +| 94588| John V| NULL| +| 94588| Zen Hui| 50| +| 94588| Dan Li| 18| ++-------+---------+-----+ + +-- Using ROW FORMAT DELIMITED +SELECT TRANSFORM(name, age) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' + USING 'cat' AS (name_age string) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY '@' + LINES TERMINATED BY '\n' + NULL DEFINED AS 'NULL' +FROM person; ++---------------+ +| name_age| ++---------------+ +| Anil K,27| +| John V,null| +| ryan B.,18| +| David K,42| +| Zen Hui,50| +| Dan Li,18| +| Lalit B.,null| ++---------------+ + +-- Using Hive Serde +SELECT TRANSFORM(zip_code, name, age) + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + WITH SERDEPROPERTIES ( + 'field.delim' = '\t' + ) + USING 'cat' AS (a STRING, b STRING, c STRING) + ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' + WITH SERDEPROPERTIES ( + 'field.delim' = '\t' + ) +FROM person +WHERE zip_code > 94511; ++-------+---------+-----+ +| a | b| c| ++-------+---------+-----+ +| 94588| Anil K| 27| +| 94588| John V| NULL| +| 94588| Zen Hui| 50| +| 94588| Dan Li| 18| ++-------+---------+-----+ + +-- Schema-less mode +SELECT TRANSFORM(zip_code, name, age) + USING 'cat' +FROM person +WHERE zip_code > 94500; ++-------+---------------------+ +| key| value| ++-------+---------------------+ +| 94588| Anil K 27| +| 94588| John V \N| +| 94511| Aryan B. 18| +| 94511| David K 42| +| 94588| Zen Hui 50| +| 94588| Dan Li 18| +| 94511| Lalit B. \N| ++-------+---------------------+ +``` + +### Related Statements + +* [SELECT Main](sql-ref-syntax-qry-select.html) +* [WHERE Clause](sql-ref-syntax-qry-select-where.html) +* [GROUP BY Clause](sql-ref-syntax-qry-select-groupby.html) +* [HAVING Clause](sql-ref-syntax-qry-select-having.html) +* [ORDER BY Clause](sql-ref-syntax-qry-select-orderby.html) +* [SORT BY Clause](sql-ref-syntax-qry-select-sortby.html) +* [DISTRIBUTE BY Clause](sql-ref-syntax-qry-select-distribute-by.html) +* [LIMIT Clause](sql-ref-syntax-qry-select-limit.html) +* [CASE Clause](sql-ref-syntax-qry-select-case.html) +* [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) +* [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) diff --git a/docs/sql-ref-syntax-qry-select.md b/docs/sql-ref-syntax-qry-select.md index 5820a5c9060e4..500eda162bf75 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 | regex_column_names ] [ , ... ] } +SELECT [ hints , ... ] [ ALL | DISTINCT ] { [ [ named_expression | regex_column_names ] [ , ... ] | TRANSFORM (...) ] } FROM { from_item [ , ... ] } [ PIVOT clause ] [ LATERAL VIEW clause ] [ ... ] @@ -164,6 +164,10 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { [ named_expression | regex_column_na ) ``` +* **TRANSFORM** + + Specifies a hive-style transform query specification to transform the input by forking and running user-specified command or script. + ### Related Statements * [WHERE Clause](sql-ref-syntax-qry-select-where.html) @@ -187,3 +191,4 @@ SELECT [ hints , ... ] [ ALL | DISTINCT ] { [ named_expression | regex_column_na * [CASE Clause](sql-ref-syntax-qry-select-case.html) * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) +* [TRANSFORM Clause](sql-ref-syntax-qry-select-transform.html) diff --git a/docs/sql-ref-syntax-qry.md b/docs/sql-ref-syntax-qry.md index 6751b90e12443..9fb62dfd548ed 100644 --- a/docs/sql-ref-syntax-qry.md +++ b/docs/sql-ref-syntax-qry.md @@ -49,4 +49,5 @@ ability to generate logical and physical plan for a given query using * [CASE Clause](sql-ref-syntax-qry-select-case.html) * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) + * [TRANSFORM Clause](sql-ref-syntax-qry-select-transform.html) * [EXPLAIN Statement](sql-ref-syntax-qry-explain.html) diff --git a/docs/sql-ref-syntax.md b/docs/sql-ref-syntax.md index f3d35b57d90cd..f0b993601876c 100644 --- a/docs/sql-ref-syntax.md +++ b/docs/sql-ref-syntax.md @@ -70,6 +70,7 @@ Spark SQL is Apache Spark's module for working with structured data. The SQL Syn * [CASE Clause](sql-ref-syntax-qry-select-case.html) * [PIVOT Clause](sql-ref-syntax-qry-select-pivot.html) * [LATERAL VIEW Clause](sql-ref-syntax-qry-select-lateral-view.html) + * [TRANSFORM Clause](sql-ref-syntax-qry-select-transform.html) * [EXPLAIN](sql-ref-syntax-qry-explain.html) ### Auxiliary Statements diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 5336695478c14..35b4f4366b820 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -140,7 +140,7 @@ df = spark \ df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") # Subscribe to 1 topic, with headers -val df = spark \ +df = spark \ .readStream \ .format("kafka") \ .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 7a0f17f5ad13f..831b4f8523828 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -181,7 +181,7 @@ The master URL passed to Spark can be in one of the following formats: The cluster location will be found based on the HADOOP_CONF_DIR or YARN_CONF_DIR variable. k8s://HOST:PORT Connect to a Kubernetes cluster in - cluster mode. Client mode is currently unsupported and will be supported in future releases. + client or cluster mode depending on the value of --deploy-mode. The HOST and PORT refer to the Kubernetes API Server. It connects using TLS by default. In order to force it to use an unsecured connection, you can use k8s://http://HOST:PORT. diff --git a/examples/pom.xml b/examples/pom.xml index 5c6b53f691b41..c9bcfe5d9e753 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index 46e740d78bffb..cb34db1ae8dfc 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -204,15 +204,15 @@ private static void runBasicDataSourceExample(SparkSession spark) { .save("namesPartByColor.parquet"); // $example off:write_partitioning$ // $example on:write_partition_and_bucket$ - peopleDF + usersDF .write() .partitionBy("favorite_color") .bucketBy(42, "name") - .saveAsTable("people_partitioned_bucketed"); + .saveAsTable("users_partitioned_bucketed"); // $example off:write_partition_and_bucket$ spark.sql("DROP TABLE IF EXISTS people_bucketed"); - spark.sql("DROP TABLE IF EXISTS people_partitioned_bucketed"); + spark.sql("DROP TABLE IF EXISTS users_partitioned_bucketed"); } private static void runBasicParquetExample(SparkSession spark) { diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java index 8605852d0881c..86a9045d8a2c4 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQLExample.java @@ -65,7 +65,7 @@ public class JavaSparkSQLExample { // $example on:create_ds$ public static class Person implements Serializable { private String name; - private int age; + private long age; public String getName() { return name; @@ -75,11 +75,11 @@ public void setName(String name) { this.name = name; } - public int getAge() { + public long getAge() { return age; } - public void setAge(int age) { + public void setAge(long age) { this.age = age; } } @@ -225,11 +225,11 @@ private static void runDatasetCreationExample(SparkSession spark) { // +---+----+ // Encoders for most common types are provided in class Encoders - Encoder integerEncoder = Encoders.INT(); - Dataset primitiveDS = spark.createDataset(Arrays.asList(1, 2, 3), integerEncoder); - Dataset transformedDS = primitiveDS.map( - (MapFunction) value -> value + 1, - integerEncoder); + Encoder longEncoder = Encoders.LONG(); + Dataset primitiveDS = spark.createDataset(Arrays.asList(1L, 2L, 3L), longEncoder); + Dataset transformedDS = primitiveDS.map( + (MapFunction) value -> value + 1L, + longEncoder); transformedDS.collect(); // Returns [2, 3, 4] // DataFrames can be converted to a Dataset by providing a class. Mapping based on name diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index 8c146ba0c9455..3bc31a0438342 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -104,7 +104,7 @@ def basic_datasource_example(spark): .write .partitionBy("favorite_color") .bucketBy(42, "name") - .saveAsTable("people_partitioned_bucketed")) + .saveAsTable("users_partitioned_bucketed")) # $example off:write_partition_and_bucket$ # $example on:manual_load_options$ @@ -135,7 +135,7 @@ def basic_datasource_example(spark): # $example off:direct_sql$ spark.sql("DROP TABLE IF EXISTS people_bucketed") - spark.sql("DROP TABLE IF EXISTS people_partitioned_bucketed") + spark.sql("DROP TABLE IF EXISTS users_partitioned_bucketed") def parquet_example(spark): diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 76eabe642ba84..c678e0c689a05 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala new file mode 100644 index 0000000000000..98a7190ba984e --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroScanSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.avro + +import org.apache.spark.sql.FileScanSuiteBase +import org.apache.spark.sql.v2.avro.AvroScan + +class AvroScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("AvroScan", + (s, fi, ds, rds, rps, f, o, pf, df) => AvroScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty)) + + run(scanBuilders) +} diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala index fde858e0a7419..b6efcc36a3139 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.types._ * {{{ * To run this benchmark: * 1. without sbt: bin/spark-submit --class - * --jars ,, + * --jars ,,, * 2. build/sbt "avro/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "avro/test:runMain " * Results will be written to "benchmarks/AvroReadBenchmark-results.txt". diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 3767a93ba7c78..0a3e6837421bd 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala index 5cbe6fab186a5..f79809f355c0f 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2KrbIntegrationSuite.scala @@ -81,7 +81,7 @@ class DB2KrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected def setAuthentication(keytabFile: String, principal: String): Unit = { val config = new SecureConnectionProvider.JDBCConfiguration( - Configuration.getConfiguration, "JaasClient", keytabFile, principal) + Configuration.getConfiguration, "JaasClient", keytabFile, principal, true) Configuration.setConfiguration(config) } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala index c20c006f3b8b6..3865f9100b846 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DockerKrbJDBCIntegrationSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.util.{SecurityUtils, Utils} abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite { private var kdc: MiniKdc = _ + private val KRB5_CONF_PROP = "java.security.krb5.conf" protected var entryPointDir: File = _ protected var initDbDir: File = _ protected val userName: String @@ -160,4 +161,41 @@ abstract class DockerKrbJDBCIntegrationSuite extends DockerJDBCIntegrationSuite assert(rows(0).getString(0) === "foo") assert(rows(0).getString(1) === "bar") } + + test("SPARK-35226: JDBCOption should accept refreshKrb5Config parameter") { + // This makes sure Spark must do authentication + Configuration.setConfiguration(null) + withTempDir { dir => + val dummyKrb5Conf = File.createTempFile("dummy", "krb5.conf", dir) + val origKrb5Conf = sys.props(KRB5_CONF_PROP) + try { + // Set dummy krb5.conf and refresh config so this assertion is expected to fail. + // The thrown exception is dependent on the actual JDBC driver class. + intercept[Exception] { + sys.props(KRB5_CONF_PROP) = dummyKrb5Conf.getAbsolutePath + spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("keytab", keytabFullPath) + .option("principal", principal) + .option("refreshKrb5Config", "true") + .option("dbtable", "bar") + .load() + } + + sys.props(KRB5_CONF_PROP) = origKrb5Conf + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("keytab", keytabFullPath) + .option("principal", principal) + .option("refreshKrb5Config", "true") + .option("dbtable", "bar") + .load() + val result = df.collect().map(_.getString(0)) + assert(result.length === 1) + assert(result(0) === "hello") + } finally { + sys.props(KRB5_CONF_PROP) = origKrb5Conf + } + } + } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala index 59a6f530afd7e..9b653f81afe9f 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MariaDBKrbIntegrationSuite.scala @@ -68,7 +68,7 @@ class MariaDBKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected def setAuthentication(keytabFile: String, principal: String): Unit = { val config = new SecureConnectionProvider.JDBCConfiguration( - Configuration.getConfiguration, "Krb5ConnectorContext", keytabFile, principal) + Configuration.getConfiguration, "Krb5ConnectorContext", keytabFile, principal, true) Configuration.setConfiguration(config) } } diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala index 984890f22f492..1198ba8a3e465 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresKrbIntegrationSuite.scala @@ -61,7 +61,7 @@ class PostgresKrbIntegrationSuite extends DockerKrbJDBCIntegrationSuite { override protected def setAuthentication(keytabFile: String, principal: String): Unit = { val config = new SecureConnectionProvider.JDBCConfiguration( - Configuration.getConfiguration, "pgjdbc", keytabFile, principal) + Configuration.getConfiguration, "pgjdbc", keytabFile, principal, true) Configuration.setConfiguration(config) } } diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index af9bba3eaac03..0376d0672dd92 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 2f0c3831a33e8..79778fb4ea066 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/kafka-0-10-token-provider/pom.xml b/external/kafka-0-10-token-provider/pom.xml index 173854cc7671a..bdae71dd9a5d3 100644 --- a/external/kafka-0-10-token-provider/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 7c5ec641a0c7a..77aa1b5afba1f 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 3d79fd0e687c1..655ec2680fcb5 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 81e5119d0da5e..7d27794b982da 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 1ecf9c00516d9..facef36532c25 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9eacb07ce60fd..ed81876459a84 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 41a9fa0b77b27..d66aae994f71d 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 44daf279cb40e..2097cbd9890cb 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 74d046024961e..efd5ddaa1db05 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index e21ae43996ae3..5ee35f537d5f5 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala index d0b6ab1ef2cbc..71403acc91b55 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Imputer.scala @@ -97,7 +97,7 @@ private[feature] trait ImputerParams extends Params with HasInputCol with HasInp } /** - * Imputation estimator for completing missing values, either using the mean or the median + * Imputation estimator for completing missing values, using the mean, median or mode * of the columns in which the missing values are located. The input columns should be of * numeric type. Currently Imputer does not support categorical features * (SPARK-15041) and possibly creates incorrect values for a categorical feature. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 68f6ed4281dea..92ae6b5bf8a03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -1313,8 +1313,10 @@ private[spark] abstract class SerDeBase { def dumps(obj: AnyRef): Array[Byte] = { obj match { // Pickler in Python side cannot deserialize Scala Array normally. See SPARK-12834. - case array: Array[_] => new Pickler().dumps(array.toSeq.asJava) - case _ => new Pickler().dumps(obj) + case array: Array[_] => new Pickler(/* useMemo = */ true, + /* valueCompare = */ false).dumps(array.toSeq.asJava) + case _ => new Pickler(/* useMemo = */ true, + /* valueCompare = */ false).dumps(obj) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala index 3caa8f6d5b1e5..e748e3288b643 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -24,7 +24,9 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder * Serialization benchmark for VectorUDT. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars * 2. build/sbt "mllib/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "mllib/test:runMain " * Results will be written to "benchmarks/UDTSerializationBenchmark-results.txt". diff --git a/pom.xml b/pom.xml index 36d87a50ab1e0..03c37aac74036 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 pom Spark Project Parent POM http://spark.apache.org/ @@ -137,7 +137,7 @@ 10.12.1.1 1.10.1 1.5.12 - 9.4.36.v20210114 + 9.4.40.v20210413 4.0.3 0.9.5 2.4.0 @@ -199,6 +199,7 @@ 2.8 1.8 1.1.0 + 1.2 + 2.12.10 + diff --git a/python/docs/source/development/contributing.rst b/python/docs/source/development/contributing.rst index a41b8a1a1de9e..aa2e9c23f573f 100644 --- a/python/docs/source/development/contributing.rst +++ b/python/docs/source/development/contributing.rst @@ -21,14 +21,14 @@ Contributing to PySpark There are many types of contribution, for example, helping other users, testing releases, reviewing changes, documentation contribution, bug reporting, JIRA maintenance, code changes, etc. -These are documented at `the general guidelines `_. +These are documented at `the general guidelines `_. This page focuses on PySpark and includes additional details specifically for PySpark. Contributing by Testing Releases -------------------------------- -Before the official release, PySpark release candidates are shared in the `dev@spark.apache.org `_ mailing list to vote on. +Before the official release, PySpark release candidates are shared in the `dev@spark.apache.org `_ mailing list to vote on. This release candidates can be easily installed via pip. For example, in case of Spark 3.0.0 RC1, you can install as below: .. code-block:: bash @@ -71,7 +71,7 @@ under ``python/docs/source/reference``. Otherwise, they would not be documented Preparing to Contribute Code Changes ------------------------------------ -Before starting to work on codes in PySpark, it is recommended to read `the general guidelines `_. +Before starting to work on codes in PySpark, it is recommended to read `the general guidelines `_. There are a couple of additional notes to keep in mind when contributing to codes in PySpark: * Be Pythonic. diff --git a/python/docs/source/getting_started/index.rst b/python/docs/source/getting_started/index.rst index 38b9c935fc623..f6d7a92ced03f 100644 --- a/python/docs/source/getting_started/index.rst +++ b/python/docs/source/getting_started/index.rst @@ -22,8 +22,8 @@ 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 `_. +`Quick Start `_ in Programming Guides +at `the Spark documentation `_. .. toctree:: :maxdepth: 2 diff --git a/python/docs/source/migration_guide/index.rst b/python/docs/source/migration_guide/index.rst index 88e768dc464df..d309d44780d1d 100644 --- a/python/docs/source/migration_guide/index.rst +++ b/python/docs/source/migration_guide/index.rst @@ -36,8 +36,8 @@ 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: -- `Migration Guide: Spark Core `_ -- `Migration Guide: SQL, Datasets and DataFrame `_ -- `Migration Guide: Structured Streaming `_ -- `Migration Guide: MLlib (Machine Learning) `_ +- `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/user_guide/arrow_pandas.rst b/python/docs/source/user_guide/arrow_pandas.rst index 91d8155523391..12b772f62abe2 100644 --- a/python/docs/source/user_guide/arrow_pandas.rst +++ b/python/docs/source/user_guide/arrow_pandas.rst @@ -408,5 +408,5 @@ This will instruct PyArrow >= 0.15.0 to use the legacy IPC format with the older is in Spark 2.3.x and 2.4.x. Not setting this environment variable will lead to a similar error as described in `SPARK-29367 `_ when running ``pandas_udf``\s or :meth:`DataFrame.toPandas` with Arrow enabled. More information about the Arrow IPC change can -be read on the Arrow 0.15.0 release `blog `_. +be read on the Arrow 0.15.0 release `blog `_. diff --git a/python/docs/source/user_guide/index.rst b/python/docs/source/user_guide/index.rst index 704156b11d985..3897ab2ea9086 100644 --- a/python/docs/source/user_guide/index.rst +++ b/python/docs/source/user_guide/index.rst @@ -30,11 +30,11 @@ This page is the guide for PySpark users which contains PySpark specific topics. There are more guides shared with other languages in Programming Guides -at `the Spark documentation `_. +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 `_ +- `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/__init__.py b/python/pyspark/__init__.py index 54b7f8373beee..8e36d9fc941af 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -60,7 +60,7 @@ from pyspark.serializers import MarshalSerializer, PickleSerializer from pyspark.taskcontext import TaskContext, BarrierTaskContext, BarrierTaskInfo from pyspark.profiler import Profiler, BasicProfiler -from pyspark.version import __version__ # noqa: F401 +from pyspark.version import __version__ from pyspark._globals import _NoValue # noqa: F401 @@ -125,4 +125,5 @@ def wrapper(self, *args, **kwargs): "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", "StatusTracker", "SparkJobInfo", "SparkStageInfo", "Profiler", "BasicProfiler", "TaskContext", "RDDBarrier", "BarrierTaskContext", "BarrierTaskInfo", "InheritableThread", + "__version__", ] diff --git a/python/pyspark/__init__.pyi b/python/pyspark/__init__.pyi index ef07c32b1db7b..852de35496046 100644 --- a/python/pyspark/__init__.pyi +++ b/python/pyspark/__init__.pyi @@ -52,6 +52,7 @@ from pyspark.taskcontext import ( # noqa: F401 TaskContext as TaskContext, ) from pyspark.util import InheritableThread as InheritableThread # noqa: F401 +from pyspark.version import __version__ as __version__ # Compatibility imports from pyspark.sql import ( # noqa: F401 diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 0553a61c6c771..6b8951d8eaefa 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -40,7 +40,7 @@ from pyspark.ml.wrapper import JavaParams, \ JavaPredictor, JavaPredictionModel, JavaWrapper from pyspark.ml.common import inherit_doc -from pyspark.ml.linalg import Vectors +from pyspark.ml.linalg import Vectors, VectorUDT from pyspark.sql import DataFrame from pyspark.sql.functions import udf, when from pyspark.sql.types import ArrayType, DoubleType @@ -738,7 +738,7 @@ def intercept(self): @since("3.1.0") def summary(self): """ - Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + Gets summary (accuracy/precision/recall, objective history, total iterations) of model trained on the training set. An exception is thrown if `trainingSummary is None`. """ if self.hasSummary: @@ -1237,7 +1237,7 @@ def interceptVector(self): @since("2.0.0") def summary(self): """ - Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + Gets summary (accuracy/precision/recall, objective history, total iterations) of model trained on the training set. An exception is thrown if `trainingSummary is None`. """ if self.hasSummary: @@ -1843,7 +1843,7 @@ def trees(self): @since("3.1.0") def summary(self): """ - Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + Gets summary (accuracy/precision/recall, objective history, total iterations) of model trained on the training set. An exception is thrown if `trainingSummary is None`. """ if self.hasSummary: @@ -2700,7 +2700,7 @@ def weights(self): @since("3.1.0") def summary(self): """ - Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + Gets summary (accuracy/precision/recall, objective history, total iterations) of model trained on the training set. An exception is thrown if `trainingSummary is None`. """ if self.hasSummary: @@ -3151,7 +3151,7 @@ def func(predictions): predArray.append(x) return Vectors.dense(predArray) - rawPredictionUDF = udf(func) + rawPredictionUDF = udf(func, VectorUDT()) aggregatedDataset = aggregatedDataset.withColumn( self.getRawPredictionCol(), rawPredictionUDF(aggregatedDataset[accColName])) @@ -3500,7 +3500,7 @@ def factors(self): @since("3.1.0") def summary(self): """ - Gets summary (e.g. accuracy/precision/recall, objective history, total iterations) of model + Gets summary (accuracy/precision/recall, objective history, total iterations) of model trained on the training set. An exception is thrown if `trainingSummary is None`. """ if self.hasSummary: diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 54c1a431671a3..763fea3d7cd63 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -189,7 +189,7 @@ def gaussiansDF(self): @since("2.1.0") def summary(self): """ - Gets summary (e.g. cluster assignments, cluster sizes) of the model trained on the + Gets summary (cluster assignments, cluster sizes) of the model trained on the training set. An exception is thrown if no summary exists. """ if self.hasSummary: @@ -560,7 +560,7 @@ def clusterCenters(self): @since("2.1.0") def summary(self): """ - Gets summary (e.g. cluster assignments, cluster sizes) of the model trained on the + Gets summary (cluster assignments, cluster sizes) of the model trained on the training set. An exception is thrown if no summary exists. """ if self.hasSummary: @@ -828,7 +828,7 @@ def computeCost(self, dataset): @since("2.1.0") def summary(self): """ - Gets summary (e.g. cluster assignments, cluster sizes) of the model trained on the + Gets summary (cluster assignments, cluster sizes) of the model trained on the training set. An exception is thrown if no summary exists. """ if self.hasSummary: diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 4e8b8b43fbbd4..cd0e287a3ac08 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1536,7 +1536,7 @@ def getMissingValue(self): @inherit_doc class Imputer(JavaEstimator, _ImputerParams, JavaMLReadable, JavaMLWritable): """ - Imputation estimator for completing missing values, either using the mean or the median + Imputation estimator for completing missing values, using the mean, median or mode of the columns in which the missing values are located. The input columns should be of numeric type. Currently Imputer does not support categorical features and possibly creates incorrect values for a categorical feature. diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index d37654a7388f5..2ba3048c79ac4 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -180,10 +180,8 @@ class LinearRegression(_JavaRegressor, _LinearRegressionParams, JavaMLWritable, >>> test1 = spark.createDataFrame([(Vectors.sparse(1, [0], [1.0]),)], ["features"]) >>> abs(model.transform(test1).head().newPrediction - 1.0) < 0.001 True - >>> lr.setParams("vector") - Traceback (most recent call last): - ... - TypeError: Method setParams forces keyword arguments. + >>> lr.setParams(featuresCol="vector") + LinearRegression... >>> lr_path = temp_path + "/lr" >>> lr.save(lr_path) >>> lr2 = LinearRegression.load(lr_path) @@ -350,7 +348,7 @@ def scale(self): @since("2.0.0") def summary(self): """ - Gets summary (e.g. residuals, mse, r-squared ) of model on + Gets summary (residuals, MSE, r-squared ) of model on training set. An exception is thrown if `trainingSummary is None`. """ @@ -2272,7 +2270,7 @@ def intercept(self): @since("2.0.0") def summary(self): """ - Gets summary (e.g. residuals, deviance, pValues) of model on + Gets summary (residuals, deviance, p-values) of model on training set. An exception is thrown if `trainingSummary is None`. """ diff --git a/python/pyspark/ml/tests/test_algorithms.py b/python/pyspark/ml/tests/test_algorithms.py index 50475210607c8..35ce48b926663 100644 --- a/python/pyspark/ml/tests/test_algorithms.py +++ b/python/pyspark/ml/tests/test_algorithms.py @@ -25,7 +25,7 @@ MultilayerPerceptronClassifier, OneVsRest from pyspark.ml.clustering import DistributedLDAModel, KMeans, LocalLDAModel, LDA, LDAModel from pyspark.ml.fpm import FPGrowth -from pyspark.ml.linalg import Matrices, Vectors +from pyspark.ml.linalg import Matrices, Vectors, DenseVector from pyspark.ml.recommendation import ALS from pyspark.ml.regression import GeneralizedLinearRegression, LinearRegression from pyspark.sql import Row @@ -116,6 +116,18 @@ def test_output_columns(self): output = model.transform(df) self.assertEqual(output.columns, ["label", "features", "rawPrediction", "prediction"]) + def test_raw_prediction_column_is_of_vector_type(self): + # SPARK-35142: `OneVsRestModel` outputs raw prediction as a string column + df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), + (1.0, Vectors.sparse(2, [], [])), + (2.0, Vectors.dense(0.5, 0.5))], + ["label", "features"]) + lr = LogisticRegression(maxIter=5, regParam=0.01) + ovr = OneVsRest(classifier=lr, parallelism=1) + model = ovr.fit(df) + row = model.transform(df).head() + self.assertIsInstance(row["rawPrediction"], DenseVector) + def test_parallelism_does_not_change_output(self): df = self.spark.createDataFrame([(0.0, Vectors.dense(1.0, 0.8)), (1.0, Vectors.sparse(2, [], [])), diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 156e7f0fe65e6..21d2725791407 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -152,7 +152,7 @@ class GeneralMLWriter(MLWriter): def format(self, source): """ - Specifies the format of ML export (e.g. "pmml", "internal", or the fully qualified class + Specifies the format of ML export ("pmml", "internal", or the fully qualified class name for export). """ self.source = source @@ -202,7 +202,7 @@ def __init__(self, instance): def format(self, source): """ - Specifies the format of ML export (e.g. "pmml", "internal", or the fully qualified class + Specifies the format of ML export ("pmml", "internal", or the fully qualified class name for export). """ self._jwrite.format(source) diff --git a/python/pyspark/sql/catalog.pyi b/python/pyspark/sql/catalog.pyi index 86263fff63ce8..cb5436ddf77ea 100644 --- a/python/pyspark/sql/catalog.pyi +++ b/python/pyspark/sql/catalog.pyi @@ -17,6 +17,7 @@ # under the License. from typing import Any, Callable, List, Optional +from pyspark.sql._typing import UserDefinedFunctionLike from pyspark.sql.dataframe import DataFrame from pyspark.sql.session import SparkSession from pyspark.sql.types import DataType, StructType @@ -53,7 +54,7 @@ class Catalog: def dropGlobalTempView(self, viewName: str) -> None: ... def registerFunction( self, name: str, f: Callable[..., Any], returnType: DataType = ... - ) -> None: ... + ) -> UserDefinedFunctionLike: ... def isCached(self, tableName: str) -> bool: ... def cacheTable(self, tableName: str) -> None: ... def uncacheTable(self, tableName: str) -> None: ... diff --git a/python/pyspark/sql/column.pyi b/python/pyspark/sql/column.pyi index 1f63e65b3de81..36c1bcc1961a8 100644 --- a/python/pyspark/sql/column.pyi +++ b/python/pyspark/sql/column.pyi @@ -115,3 +115,4 @@ class Column: def over(self, window: WindowSpec) -> Column: ... def __nonzero__(self) -> None: ... def __bool__(self) -> None: ... + def contains(self, item: Any) -> Column: ... diff --git a/python/pyspark/sql/conf.pyi b/python/pyspark/sql/conf.pyi index 304dfcb3f9e53..3e88f845a1165 100644 --- a/python/pyspark/sql/conf.pyi +++ b/python/pyspark/sql/conf.pyi @@ -21,7 +21,7 @@ from py4j.java_gateway import JavaObject # type: ignore[import] class RuntimeConfig: def __init__(self, jconf: JavaObject) -> None: ... - def set(self, key: str, value: str) -> str: ... + def set(self, key: str, value: str) -> None: ... def get(self, key: str, default: Optional[str] = ...) -> str: ... def unset(self, key: str) -> None: ... def isModifiable(self, key: str) -> bool: ... diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 5071240a511a6..ad77ce61d4679 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -298,7 +298,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr Parameters ---------- data : :class:`RDD` or iterable - an RDD of any kind of SQL data representation(e.g. :class:`Row`, + an RDD of any kind of SQL data representation (:class:`Row`, :class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, or :class:`pandas.DataFrame`. schema : :class:`pyspark.sql.types.DataType`, str or list, optional diff --git a/python/pyspark/sql/context.pyi b/python/pyspark/sql/context.pyi index 915a0fe1f6709..e8b61c8d59ba9 100644 --- a/python/pyspark/sql/context.pyi +++ b/python/pyspark/sql/context.pyi @@ -15,6 +15,7 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +from pyspark.sql._typing import UserDefinedFunctionLike from typing import overload from typing import Any, Callable, Iterable, List, Optional, Tuple, TypeVar, Union @@ -63,7 +64,7 @@ class SQLContext: ) -> DataFrame: ... def registerFunction( self, name: str, f: Callable[..., Any], returnType: DataType = ... - ) -> None: ... + ) -> UserDefinedFunctionLike: ... def registerJavaFunction( self, name: str, javaClassName: str, returnType: Optional[DataType] = ... ) -> None: ... diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 631347455a204..308509248f07d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -586,7 +586,7 @@ def withWatermark(self, eventTime, delayThreshold): Parameters ---------- - eventTime : str or :class:`Column` + eventTime : str the name of the column that contains the event time of the row. delayThreshold : str the minimum delay to wait to data to arrive late, relative to the diff --git a/python/pyspark/sql/dataframe.pyi b/python/pyspark/sql/dataframe.pyi index 1351c59470c9d..af1bac61c2291 100644 --- a/python/pyspark/sql/dataframe.pyi +++ b/python/pyspark/sql/dataframe.pyi @@ -85,7 +85,7 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin): def checkpoint(self, eager: bool = ...) -> DataFrame: ... def localCheckpoint(self, eager: bool = ...) -> DataFrame: ... def withWatermark( - self, eventTime: ColumnOrName, delayThreshold: str + self, eventTime: str, delayThreshold: str ) -> DataFrame: ... def hint(self, name: str, *parameters: Union[PrimitiveType, List[PrimitiveType]]) -> DataFrame: ... def count(self) -> int: ... diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 51ab9c12317fa..2f1857d3172ca 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -4153,7 +4153,10 @@ def _create_lambda(f): argnames = ["x", "y", "z"] args = [ - _unresolved_named_lambda_variable(arg) for arg in argnames[: len(parameters)] + _unresolved_named_lambda_variable( + expressions.UnresolvedNamedLambdaVariable.freshVarName(arg) + ) + for arg in argnames[: len(parameters)] ] result = f(*args) diff --git a/python/pyspark/sql/functions.pyi b/python/pyspark/sql/functions.pyi index f1043a01a8f71..5fec6fd9fb6eb 100644 --- a/python/pyspark/sql/functions.pyi +++ b/python/pyspark/sql/functions.pyi @@ -56,7 +56,7 @@ def monotonically_increasing_id() -> Column: ... def nanvl(col1: ColumnOrName, col2: ColumnOrName) -> Column: ... def percentile_approx( col: ColumnOrName, - percentage: Union[Column, float, List[float]], + percentage: Union[Column, float, List[float], tuple[float]], accuracy: Union[Column, float] = ..., ) -> Column: ... def rand(seed: Optional[int] = ...) -> Column: ... diff --git a/python/pyspark/sql/pandas/utils.py b/python/pyspark/sql/pandas/utils.py index 9b97676341bcd..b22603a779701 100644 --- a/python/pyspark/sql/pandas/utils.py +++ b/python/pyspark/sql/pandas/utils.py @@ -26,11 +26,12 @@ def require_minimum_pandas_version(): try: import pandas have_pandas = True - except ImportError: + except ImportError as error: have_pandas = False + raised_error = error if not have_pandas: raise ImportError("Pandas >= %s must be installed; however, " - "it was not found." % minimum_pandas_version) + "it was not found." % minimum_pandas_version) from raised_error if LooseVersion(pandas.__version__) < LooseVersion(minimum_pandas_version): raise ImportError("Pandas >= %s must be installed; however, " "your version was %s." % (minimum_pandas_version, pandas.__version__)) @@ -47,11 +48,12 @@ def require_minimum_pyarrow_version(): try: import pyarrow have_arrow = True - except ImportError: + except ImportError as error: have_arrow = False + raised_error = error if not have_arrow: raise ImportError("PyArrow >= %s must be installed; however, " - "it was not found." % minimum_pyarrow_version) + "it was not found." % minimum_pyarrow_version) from raised_error if LooseVersion(pyarrow.__version__) < LooseVersion(minimum_pyarrow_version): raise ImportError("PyArrow >= %s must be installed; however, " "your version was %s." % (minimum_pyarrow_version, pyarrow.__version__)) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index d120daa5a9434..dd04e0bc92bee 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -701,7 +701,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non * ``STOP_AT_DELIMITER``: If unescaped quotes are found in the input, consider the value as an unquoted value. This will make the parser accumulate all characters until the delimiter or a line ending is found in the input. - * ``STOP_AT_DELIMITER``: If unescaped quotes are found in the input, the content parsed + * ``SKIP_VALUE``: If unescaped quotes are found in the input, the content parsed for the given value will be skipped and the value set in nullValue will be produced instead. * ``RAISE_ERROR``: If unescaped quotes are found in the input, a TextParsingException @@ -1196,7 +1196,8 @@ def json(self, path, mode=None, compression=None, dateFormat=None, timestampForm encoding : str, optional specifies encoding (charset) of saved json files. If None is set, the default UTF-8 charset will be used. - lineSep : str, optional defines the line separator that should be used for writing. If None is + lineSep : str, optional + defines the line separator that should be used for writing. If None is set, it uses the default value, ``\\n``. ignoreNullFields : str or bool, optional Whether to ignore null fields when generating JSON objects. diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 4db958ad2e0c4..57c680fd0496f 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -580,7 +580,7 @@ def createDataFrame(self, data, schema=None, samplingRatio=None, verifySchema=Tr Parameters ---------- data : :class:`RDD` or iterable - an RDD of any kind of SQL data representation(e.g. :class:`Row`, + an RDD of any kind of SQL data representation (:class:`Row`, :class:`tuple`, ``int``, ``boolean``, etc.), or :class:`list`, or :class:`pandas.DataFrame`. schema : :class:`pyspark.sql.types.DataType`, str or list, optional diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 51941a6269074..3ab72ee6a6ee9 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -915,7 +915,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non * ``STOP_AT_DELIMITER``: If unescaped quotes are found in the input, consider the value as an unquoted value. This will make the parser accumulate all characters until the delimiter or a line ending is found in the input. - * ``STOP_AT_DELIMITER``: If unescaped quotes are found in the input, the content parsed + * ``SKIP_VALUE``: If unescaped quotes are found in the input, the content parsed for the given value will be skipped and the value set in nullValue will be produced instead. * ``RAISE_ERROR``: If unescaped quotes are found in the input, a TextParsingException diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 053164a792f54..8ccc051c77f32 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -491,6 +491,28 @@ def test_higher_order_function_failures(self): with self.assertRaises(ValueError): transform(col("foo"), lambda x: 1) + def test_nested_higher_order_function(self): + # SPARK-35382: lambda vars must be resolved properly in nested higher order functions + from pyspark.sql.functions import flatten, struct, transform + + df = self.spark.sql("SELECT array(1, 2, 3) as numbers, array('a', 'b', 'c') as letters") + + actual = df.select(flatten( + transform( + "numbers", + lambda number: transform( + "letters", + lambda letter: struct(number.alias("n"), letter.alias("l")) + ) + ) + )).first()[0] + + expected = [(1, "a"), (1, "b"), (1, "c"), + (2, "a"), (2, "b"), (2, "c"), + (3, "a"), (3, "b"), (3, "c")] + + self.assertEquals(actual, expected) + def test_window_functions(self): df = self.spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"]) w = Window.partitionBy("value").orderBy("key") diff --git a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py index b49092ed70d04..0c78855cd9123 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py +++ b/python/pyspark/sql/tests/test_pandas_udf_grouped_agg.py @@ -51,7 +51,7 @@ def python_plus_one(self): @udf('double') def plus_one(v): assert isinstance(v, (int, float)) - return v + 1 + return float(v + 1) return plus_one @property @@ -316,16 +316,18 @@ def test_complex_groupby(self): expected3 = df.groupby(df.id, df.v % 2).agg(sum(df.v)).orderBy(df.id, df.v % 2) # groupby one python UDF - result4 = df.groupby(plus_one(df.id)).agg(sum_udf(df.v)) - expected4 = df.groupby(plus_one(df.id)).agg(sum(df.v)) + result4 = df.groupby(plus_one(df.id)).agg(sum_udf(df.v)).sort('plus_one(id)') + expected4 = df.groupby(plus_one(df.id)).agg(sum(df.v)).sort('plus_one(id)') # groupby one scalar pandas UDF result5 = df.groupby(plus_two(df.id)).agg(sum_udf(df.v)).sort('sum(v)') expected5 = df.groupby(plus_two(df.id)).agg(sum(df.v)).sort('sum(v)') # groupby one expression and one python UDF - result6 = df.groupby(df.v % 2, plus_one(df.id)).agg(sum_udf(df.v)) - expected6 = df.groupby(df.v % 2, plus_one(df.id)).agg(sum(df.v)) + result6 = (df.groupby(df.v % 2, plus_one(df.id)) + .agg(sum_udf(df.v)).sort(['(v % 2)', 'plus_one(id)'])) + expected6 = (df.groupby(df.v % 2, plus_one(df.id)) + .agg(sum(df.v)).sort(['(v % 2)', 'plus_one(id)'])) # groupby one expression and one scalar pandas UDF result7 = (df.groupby(df.v % 2, plus_two(df.id)) diff --git a/python/pyspark/sql/tests/test_pandas_udf_window.py b/python/pyspark/sql/tests/test_pandas_udf_window.py index d861bcce9e8b8..18379b8a8c8c0 100644 --- a/python/pyspark/sql/tests/test_pandas_udf_window.py +++ b/python/pyspark/sql/tests/test_pandas_udf_window.py @@ -43,7 +43,11 @@ def data(self): @property def python_plus_one(self): - return udf(lambda v: v + 1, 'double') + @udf('double') + def plus_one(v): + assert isinstance(v, float) + return v + 1 + return plus_one @property def pandas_scalar_time_two(self): diff --git a/python/pyspark/sql/tests/test_udf.py b/python/pyspark/sql/tests/test_udf.py index bfc55dff94540..0d13361dcabe6 100644 --- a/python/pyspark/sql/tests/test_udf.py +++ b/python/pyspark/sql/tests/test_udf.py @@ -674,6 +674,17 @@ def test_udf_cache(self): self.assertEqual(df.select(udf(func)("id"))._jdf.queryExecution() .withCachedData().getClass().getSimpleName(), 'InMemoryRelation') + # SPARK-34545 + def test_udf_input_serialization_valuecompare_disabled(self): + def f(e): + return e[0] + + df = self.spark.createDataFrame([((1.0, 1.0), (1, 1))], ['c1', 'c2']) + result = df.select("*", udf(f, DoubleType())("c1").alias('c3'), + udf(f, IntegerType())("c2").alias('c4')) + self.assertEqual(result.collect(), + [Row(c1=Row(_1=1.0, _2=1.0), c2=Row(_1=1, _2=1), c3=1.0, c4=1)]) + class UDFInitializationTests(unittest.TestCase): def tearDown(self): diff --git a/python/pyspark/tests/test_taskcontext.py b/python/pyspark/tests/test_taskcontext.py index f6e275abfb1e7..e7161be48b55a 100644 --- a/python/pyspark/tests/test_taskcontext.py +++ b/python/pyspark/tests/test_taskcontext.py @@ -23,7 +23,7 @@ import unittest from pyspark import SparkConf, SparkContext, TaskContext, BarrierTaskContext -from pyspark.testing.utils import PySparkTestCase, SPARK_HOME +from pyspark.testing.utils import PySparkTestCase, SPARK_HOME, eventually class TaskContextTests(PySparkTestCase): @@ -244,7 +244,7 @@ def context_barrier(x): for pid in pids: self.assertTrue(pid in worker_pids) - def test_task_context_correct_with_python_worker_reuse(self): + def check_task_context_correct_with_python_worker_reuse(self): """Verify the task context correct when reused python worker""" # start a normal job first to start all workers and get all worker pids worker_pids = self.sc.parallelize(range(2), 2).map(lambda x: os.getpid()).collect() @@ -263,7 +263,6 @@ def context(iterator): # normal stage after normal stage normal_result = rdd.mapPartitions(context).collect() tps, bps, pids = zip(*normal_result) - print(tps) self.assertTrue(tps == (0, 1)) self.assertTrue(bps == (-1, -1)) for pid in pids: @@ -282,6 +281,14 @@ def context(iterator): self.assertTrue(bps == (-1, -1)) for pid in pids: self.assertTrue(pid in worker_pids) + return True + + def test_task_context_correct_with_python_worker_reuse(self): + # Retrying the check as the PIDs from Python workers might be different even + # when reusing Python workers is enabled if a Python worker is dead for some reasons + # (e.g., socket connection failure) and new Python worker is created. + eventually( + self.check_task_context_correct_with_python_worker_reuse, catch_assertions=True) def tearDown(self): self.sc.stop() diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 0fd255eb83c43..07bb4e3a68ad0 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "3.1.1" +__version__ = "3.1.2" diff --git a/repl/pom.xml b/repl/pom.xml index b125aa5709271..fbe6a84ad922f 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index af1d72fc9f744..5e997b217644a 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../../pom.xml 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 68dcef1d259a3..bec29a9ad3294 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 @@ -237,7 +237,8 @@ private[spark] object Config extends Logging { val KUBERNETES_ALLOCATION_EXECUTOR_TIMEOUT = ConfigBuilder("spark.kubernetes.allocation.executor.timeout") - .doc("Time to wait before considering a pending executor timedout.") + .doc("Time to wait before a newly created executor POD request, which does not reached " + + "the POD pending state yet, considered timedout and will be deleted.") .version("3.1.0") .timeConf(TimeUnit.MILLISECONDS) .checkValue(value => value > 0, "Allocation executor timeout must be a positive time value.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index cec8272beed57..7f34f30d59982 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -96,7 +96,7 @@ private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) val driverPort = conf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT) val driverBlockManagerPort = conf.sparkConf.getInt( DRIVER_BLOCK_MANAGER_PORT.key, - DEFAULT_BLOCKMANAGER_PORT + conf.sparkConf.getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) ) val driverUIPort = SparkUI.getUIPort(conf.sparkConf) val driverContainer = new ContainerBuilder(pod.container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 250dd8238d9ea..a0a17cecf9a8e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -44,7 +44,10 @@ private[spark] class BasicExecutorFeatureStep( .getOrElse(throw new SparkException("Must specify the executor container image")) private val blockManagerPort = kubernetesConf .sparkConf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + .getInt(BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + + require(blockManagerPort == 0 || (1024 <= blockManagerPort && blockManagerPort < 65536), + "port number must be 0 or in [1024, 65535]") private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix @@ -171,14 +174,17 @@ private[spark] class BasicExecutorFeatureStep( .replaceAll(ENV_EXECUTOR_ID, kubernetesConf.executorId)) } - val requiredPorts = Seq( - (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) - .map { case (name, port) => - new ContainerPortBuilder() - .withName(name) - .withContainerPort(port) - .build() - } + // 0 is invalid as kubernetes containerPort request, we shall leave it unmounted + val requiredPorts = if (blockManagerPort != 0) { + Seq( + (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) + .map { case (name, port) => + new ContainerPortBuilder() + .withName(name) + .withContainerPort(port) + .build() + } + } else Nil if (!isDefaultProfile) { if (pod.container != null && pod.container.getResources() != null) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index eb35de8759593..5ebd172f7dec6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import java.time.Instant import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -70,7 +70,7 @@ private[spark] class ExecutorPodsAllocator( private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) - private val driverPod = kubernetesDriverPodName + val driverPod = kubernetesDriverPodName .map(name => Option(kubernetesClient.pods() .withName(name) .get()) @@ -82,9 +82,14 @@ private[spark] class ExecutorPodsAllocator( // snapshot yet. Mapped to the (ResourceProfile id, timestamp) when they were created. private val newlyCreatedExecutors = mutable.LinkedHashMap.empty[Long, (Int, Long)] + // Executor IDs that have been requested from Kubernetes but have not been detected in any POD + // snapshot yet but already known by the scheduler backend. Mapped to the ResourceProfile id. + private val schedulerKnownNewlyCreatedExecs = mutable.LinkedHashMap.empty[Long, Int] + private val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(conf) - private val hasPendingPods = new AtomicBoolean() + // visible for tests + private[k8s] val numOutstandingPods = new AtomicInteger() private var lastSnapshot = ExecutorPodsSnapshot() @@ -93,9 +98,9 @@ private[spark] class ExecutorPodsAllocator( // if they happen to come up before the deletion takes effect. @volatile private var deletedExecutorIds = Set.empty[Long] - def start(applicationId: String): Unit = { + def start(applicationId: String, schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { snapshotsStore.addSubscriber(podAllocationDelay) { - onNewSnapshots(applicationId, _) + onNewSnapshots(applicationId, schedulerBackend, _) } } @@ -105,7 +110,7 @@ private[spark] class ExecutorPodsAllocator( totalExpectedExecutorsPerResourceProfileId.put(rp.id, numExecs) } logDebug(s"Set total expected execs to $totalExpectedExecutorsPerResourceProfileId") - if (!hasPendingPods.get()) { + if (numOutstandingPods.get() == 0) { snapshotsStore.notifySubscribers() } } @@ -114,8 +119,19 @@ private[spark] class ExecutorPodsAllocator( private def onNewSnapshots( applicationId: String, + schedulerBackend: KubernetesClusterSchedulerBackend, snapshots: Seq[ExecutorPodsSnapshot]): Unit = { - newlyCreatedExecutors --= snapshots.flatMap(_.executorPods.keys) + val k8sKnownExecIds = snapshots.flatMap(_.executorPods.keys) + newlyCreatedExecutors --= k8sKnownExecIds + schedulerKnownNewlyCreatedExecs --= k8sKnownExecIds + + // transfer the scheduler backend known executor requests from the newlyCreatedExecutors + // to the schedulerKnownNewlyCreatedExecs + val schedulerKnownExecs = schedulerBackend.getExecutorIds().map(_.toLong).toSet + schedulerKnownNewlyCreatedExecs ++= + newlyCreatedExecutors.filterKeys(schedulerKnownExecs.contains(_)).mapValues(_._1) + newlyCreatedExecutors --= schedulerKnownNewlyCreatedExecs.keySet + // For all executors we've created against the API but have not seen in a snapshot // yet - check the current time. If the current time has exceeded some threshold, // assume that the pod was either never created (the API server never properly @@ -164,15 +180,16 @@ private[spark] class ExecutorPodsAllocator( _deletedExecutorIds = _deletedExecutorIds.filter(existingExecs.contains) } + val notDeletedPods = lastSnapshot.executorPods.filterKeys(!_deletedExecutorIds.contains(_)) // Map the pods into per ResourceProfile id so we can check per ResourceProfile, // add a fast path if not using other ResourceProfiles. val rpIdToExecsAndPodState = mutable.HashMap[Int, mutable.HashMap[Long, ExecutorPodState]]() if (totalExpectedExecutorsPerResourceProfileId.size <= 1) { rpIdToExecsAndPodState(ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) = - mutable.HashMap.empty ++= lastSnapshot.executorPods + mutable.HashMap.empty ++= notDeletedPods } else { - lastSnapshot.executorPods.foreach { case (execId, execPodState) => + notDeletedPods.foreach { case (execId, execPodState) => val rpId = execPodState.pod.getMetadata.getLabels.get(SPARK_RESOURCE_PROFILE_ID_LABEL).toInt val execPods = rpIdToExecsAndPodState.getOrElseUpdate(rpId, mutable.HashMap[Long, ExecutorPodState]()) @@ -190,24 +207,33 @@ private[spark] class ExecutorPodsAllocator( case _ => false } - val currentPendingExecutors = podsForRpId.filter { + val (schedulerKnownPendingExecsForRpId, currentPendingExecutorsForRpId) = podsForRpId.filter { case (_, PodPending(_)) => true case _ => false + }.partition { case (k, _) => + schedulerKnownExecs.contains(k) } // This variable is used later to print some debug logs. It's updated when cleaning up - // excess pod requests, since currentPendingExecutors is immutable. - var knownPendingCount = currentPendingExecutors.size + // excess pod requests, since currentPendingExecutorsForRpId is immutable. + var knownPendingCount = currentPendingExecutorsForRpId.size val newlyCreatedExecutorsForRpId = newlyCreatedExecutors.filter { case (_, (waitingRpId, _)) => rpId == waitingRpId } + val schedulerKnownNewlyCreatedExecsForRpId = + schedulerKnownNewlyCreatedExecs.filter { case (_, waitingRpId) => + rpId == waitingRpId + } + if (podsForRpId.nonEmpty) { logDebug(s"ResourceProfile Id: $rpId " + s"pod allocation status: $currentRunningCount running, " + - s"${currentPendingExecutors.size} pending. " + - s"${newlyCreatedExecutorsForRpId.size} unacknowledged.") + s"${currentPendingExecutorsForRpId.size} unknown pending, " + + s"${schedulerKnownPendingExecsForRpId.size} scheduler backend known pending, " + + s"${newlyCreatedExecutorsForRpId.size} unknown newly created, " + + s"${schedulerKnownNewlyCreatedExecsForRpId.size} scheduler backend known newly created.") } // It's possible that we have outstanding pods that are outdated when dynamic allocation @@ -218,8 +244,9 @@ private[spark] class ExecutorPodsAllocator( // // TODO: with dynamic allocation off, handle edge cases if we end up with more running // executors than expected. - val knownPodCount = currentRunningCount + currentPendingExecutors.size + - newlyCreatedExecutorsForRpId.size + val knownPodCount = currentRunningCount + + currentPendingExecutorsForRpId.size + schedulerKnownPendingExecsForRpId.size + + newlyCreatedExecutorsForRpId.size + schedulerKnownNewlyCreatedExecsForRpId.size if (knownPodCount > targetNum) { val excess = knownPodCount - targetNum @@ -227,7 +254,7 @@ private[spark] class ExecutorPodsAllocator( .filter { case (_, (_, createTime)) => currentTime - createTime > executorIdleTimeout }.keys.take(excess).toList - val knownPendingToDelete = currentPendingExecutors + val knownPendingToDelete = currentPendingExecutorsForRpId .filter(x => isExecutorIdleTimedOut(x._2, currentTime)) .take(excess - newlyCreatedToDelete.size) .map { case (id, _) => id } @@ -245,7 +272,7 @@ private[spark] class ExecutorPodsAllocator( .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, toDelete.sorted.map(_.toString): _*) .delete() - newlyCreatedExecutors --= toDelete + newlyCreatedExecutors --= newlyCreatedToDelete knownPendingCount -= knownPendingToDelete.size } } @@ -276,8 +303,9 @@ private[spark] class ExecutorPodsAllocator( deletedExecutorIds = _deletedExecutorIds // Update the flag that helps the setTotalExpectedExecutors() callback avoid triggering this - // update method when not needed. - hasPendingPods.set(totalPendingCount + newlyCreatedExecutors.size > 0) + // update method when not needed. PODs known by the scheduler backend are not counted here as + // they considered running PODs and they should not block upscaling. + numOutstandingPods.set(totalPendingCount + newlyCreatedExecutors.size) } private def requestNewExecutors( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index c35a434f83ec1..887afca05cb1c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -20,11 +20,13 @@ import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import scala.concurrent.Future +import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.KubernetesClient import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.submit.KubernetesClientUtils import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO @@ -67,13 +69,14 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private def setUpExecutorConfigMap(): Unit = { + private def setUpExecutorConfigMap(driverPod: Option[Pod]): Unit = { val configMapName = KubernetesClientUtils.configMapNameExecutor val confFilesMap = KubernetesClientUtils .buildSparkConfDirFilesMap(configMapName, conf, Map.empty) val labels = Map(SPARK_APP_ID_LABEL -> applicationId(), SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) val configMap = KubernetesClientUtils.buildConfigMap(configMapName, confFilesMap, labels) + KubernetesUtils.addOwnerReference(driverPod.orNull, Seq(configMap)) kubernetesClient.configMaps().create(configMap) } @@ -93,10 +96,10 @@ private[spark] class KubernetesClusterSchedulerBackend( val initExecs = Map(defaultProfile -> initialExecutors) podAllocator.setTotalExpectedExecutors(initExecs) lifecycleEventHandler.start(this) - podAllocator.start(applicationId()) + podAllocator.start(applicationId(), this) watchEvents.start(applicationId()) pollEvents.start(applicationId()) - setUpExecutorConfigMap() + setUpExecutorConfigMap(podAllocator.driverPod) } override def stop(): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 858b4f1494b8e..f0843225ea6e2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -213,6 +213,25 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { } } + test("SPARK-35493: make spark.blockManager.port be able to be fallen back to in driver pod") { + val initPod = SparkPod.initialPod() + val sparkConf = new SparkConf() + .set(CONTAINER_IMAGE, "spark-driver:latest") + .set(BLOCK_MANAGER_PORT, 1234) + val driverConf1 = KubernetesTestConf.createDriverConf(sparkConf) + val pod1 = new BasicDriverFeatureStep(driverConf1).configurePod(initPod) + val portMap1 = + pod1.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap1(BLOCK_MANAGER_PORT_NAME) === 1234, s"fallback to $BLOCK_MANAGER_PORT.key") + + val driverConf2 = + KubernetesTestConf.createDriverConf(sparkConf.set(DRIVER_BLOCK_MANAGER_PORT, 1235)) + val pod2 = new BasicDriverFeatureStep(driverConf2).configurePod(initPod) + val portMap2 = + pod2.container.getPorts.asScala.map { cp => (cp.getName -> cp.getContainerPort) }.toMap + assert(portMap2(BLOCK_MANAGER_PORT_NAME) === 1235) + } + def containerPort(name: String, portNumber: Int): ContainerPort = new ContainerPortBuilder() .withName(name) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 66ece81aca646..df4693f313917 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -330,6 +330,35 @@ class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { SecretVolumeUtils.podHasVolume(podConfigured.pod, SPARK_CONF_VOLUME_EXEC) } + test("SPARK-35482: user correct block manager port for executor pods") { + try { + val initPod = SparkPod.initialPod() + val sm = new SecurityManager(baseConf) + val step1 = + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort1 = step1.configurePod(initPod).container.getPorts.get(0) + assert(containerPort1.getContainerPort === DEFAULT_BLOCKMANAGER_PORT, + s"should use port no. $DEFAULT_BLOCKMANAGER_PORT as default") + + baseConf.set(BLOCK_MANAGER_PORT, 12345) + val step2 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + val containerPort2 = step2.configurePod(initPod).container.getPorts.get(0) + assert(containerPort2.getContainerPort === 12345) + + baseConf.set(BLOCK_MANAGER_PORT, 1000) + val e = intercept[IllegalArgumentException] { + new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + } + assert(e.getMessage.contains("port number must be 0 or in [1024, 65535]")) + + baseConf.set(BLOCK_MANAGER_PORT, 0) + val step3 = new BasicExecutorFeatureStep(newExecutorConf(), sm, defaultProfile) + assert(step3.configurePod(initPod).container.getPorts.isEmpty, "random port") + } finally { + baseConf.remove(BLOCK_MANAGER_PORT) + } + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 349bbcd6f7883..55be80ae29c7e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -81,6 +81,9 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var executorBuilder: KubernetesExecutorBuilder = _ + @Mock + private var schedulerBackend: KubernetesClusterSchedulerBackend = _ + private var snapshotsStore: DeterministicExecutorPodsSnapshotsStore = _ private var podsAllocatorUnderTest: ExecutorPodsAllocator = _ @@ -96,12 +99,14 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) - podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) + when(schedulerBackend.getExecutorIds).thenReturn(Seq.empty) + podsAllocatorUnderTest.start(TEST_SPARK_APP_ID, schedulerBackend) } test("Initially request executors in batches. Do not request another batch if the" + " first has not finished.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (nextId <- 1 to podAllocationSize) { verify(podOperations).create(podWithAttachedContainerForId(nextId)) } @@ -111,28 +116,34 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { test("Request executors in batches. Allow another batch to be requested if" + " all pending executors start running.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> (podAllocationSize + 1))) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, never()).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) snapshotsStore.updatePod(runningExecutor(podAllocationSize)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations, times(podAllocationSize + 1)).create(any(classOf[Pod])) } test("When a current batch reaches error states immediately, re-request" + " them on the next batch.") { podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> podAllocationSize)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) for (execId <- 1 until podAllocationSize) { snapshotsStore.updatePod(runningExecutor(execId)) } val failedPod = failedExecutorWithoutDeletion(podAllocationSize) snapshotsStore.updatePod(failedPod) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(podAllocationSize + 1)) } @@ -148,9 +159,11 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1")) .thenReturn(labeledPods) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) } @@ -174,17 +187,20 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor, make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 1) verify(podOperations).create(podWithAttachedContainerForId(1)) // Mark executor as running, verify that subsequent allocation cycle is a no-op. snapshotsStore.updatePod(runningExecutor(1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(1)).create(any()) verify(podOperations, never()).delete() // Request 3 more executors, make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) verify(podOperations).create(podWithAttachedContainerForId(4)) @@ -193,6 +209,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2)) snapshotsStore.updatePod(pendingExecutor(3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) verify(podOperations, times(4)).create(any()) verify(podOperations, never()).delete() @@ -200,6 +217,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(4)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "3", "4") verify(podOperations).delete() @@ -212,6 +230,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(4)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("3")) assert(!podsAllocatorUnderTest.isDeleted("4")) } @@ -279,6 +298,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.setTime(startTime) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 5)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) verify(podOperations).create(podWithAttachedContainerForId(3)) @@ -292,16 +312,139 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Newly created executors (both acknowledged and not) are protected by executorIdleTimeout podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 0)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) verify(podOperations, never()).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") verify(podOperations, never()).delete() // Newly created executors (both acknowledged and not) are cleaned up. waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "3", "4")) snapshotsStore.notifySubscribers() - verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "1", "2", "3", "4", "5") + // SPARK-34361: even as 1, 3 and 4 are not timed out as they are considered as known PODs so + // this is why they are not counted into the outstanding PODs and /they are not removed even + // though executor 1 is still in pending state and executor 3 and 4 are new request without + // any state reported by kubernetes and all the three are already timed out + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "2", "5") verify(podOperations).delete() } + /** + * This test covers some downscaling and upscaling of dynamic allocation on kubernetes + * along with multiple resource profiles (default and rp) when some executors + * already know by the scheduler backend. + * + * Legend: + * + * N-: newly created not known by the scheduler backend + * N+: newly created known by the scheduler backend + * P- / P+ : pending (not know / known) by the scheduler backend + * D: deleted + * | default || rp | expected + * | || | outstanding + * | 1 | 2 | 3 || 4 | 5 | 6 | 7 | PODs + * ========================================================================================== + * 0) setTotalExpectedExecs with | N- | N- | N- || N- | N- | N- | N- | + * default->3, ro->4 | | | || | | | | 7 + * ------------------------------------------------------------------------------------------ + * 1) make 1 from each rp | N+ | N- | N- || N+ | N- | N- | N- | + * known by backend | | | || | | | | 5 + * ------------------------------------------------------------------------------------------- + * 2) some more backend known + pending | N+ | P+ | P- || N+ | P+ | P- | N- | 3 + * ------------------------------------------------------------------------------------------- + * 3) advance time with idle timeout | | | || | | | | + * setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 + * default->1, rp->1 | | | || | | | | + * ------------------------------------------------------------------------------------------- + * 4) setTotalExpectedExecs with | N+ | P+ | D || N+ | P+ | D | D | 0 and + * default->2, rp->2 | | | || | | | | no new POD req. + * =========================================================================================== + * + * 5) setTotalExpectedExecs with default -> 3, rp -> 3 which will lead to creation of the new + * PODs: 8 and 9 + */ + test("SPARK-34361: scheduler backend known pods with multiple resource profiles at downscaling") { + when(podOperations + .withField("status.phase", "Pending")) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabelIn(meq(SPARK_EXECUTOR_ID_LABEL), any())) + .thenReturn(podOperations) + + val startTime = Instant.now.toEpochMilli + waitForExecutorPodsClock.setTime(startTime) + + val rpb = new ResourceProfileBuilder() + val ereq = new ExecutorResourceRequests() + val treq = new TaskResourceRequests() + ereq.cores(4).memory("2g") + treq.cpus(2) + rpb.require(ereq).require(treq) + val rp = rpb.build() + + // 0) request 3 PODs for the default and 4 PODs for the other resource profile + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 4)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 7) + verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(2, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(3, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(4, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(5, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(6, rp.id)) + verify(podOperations).create(podWithAttachedContainerForId(7, rp.id)) + + // 1) make 1 POD known by the scheduler backend for each resource profile + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "4")) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5, + "scheduler backend known PODs are not outstanding") + verify(podOperations, times(7)).create(any()) + + // 2) make 1 extra POD known by the scheduler backend for each resource profile + // and make some to pending + when(schedulerBackend.getExecutorIds).thenReturn(Seq("1", "2", "4", "5")) + snapshotsStore.updatePod(pendingExecutor(2, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(3, defaultProfile.id)) + snapshotsStore.updatePod(pendingExecutor(5, rp.id)) + snapshotsStore.updatePod(pendingExecutor(6, rp.id)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) + verify(podOperations, times(7)).create(any()) + + // 3) downscale to 1 POD for default and 1 POD for the other resource profile + waitForExecutorPodsClock.advance(executorIdleTimeout * 2) + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + verify(podOperations, times(2)).delete() + assert(podsAllocatorUnderTest.isDeleted("3")) + assert(podsAllocatorUnderTest.isDeleted("6")) + assert(podsAllocatorUnderTest.isDeleted("7")) + + // 4) upscale to 2 PODs for default and 2 for the other resource profile but as there is still + // 2 PODs known by the scheduler backend there must be no new POD requested to be created + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 2, rp -> 2)) + snapshotsStore.notifySubscribers() + verify(podOperations, times(7)).create(any()) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) + verify(podOperations, times(7)).create(any()) + + // 5) requesting 1 more executor for each resource + podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 3, rp -> 3)) + snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) + verify(podOperations, times(9)).create(any()) + verify(podOperations).create(podWithAttachedContainerForId(8, defaultProfile.id)) + verify(podOperations).create(podWithAttachedContainerForId(9, rp.id)) + } + test("SPARK-33288: multiple resource profiles") { when(podOperations .withField("status.phase", "Pending")) @@ -330,6 +473,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // Target 1 executor for default profile, 2 for other profile, // make sure it's requested, even with an empty initial snapshot. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 2)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations).create(podWithAttachedContainerForId(1, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(2, rp.id)) verify(podOperations).create(podWithAttachedContainerForId(3, rp.id)) @@ -339,6 +483,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(runningExecutor(2, rp.id)) snapshotsStore.updatePod(runningExecutor(3, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(3)).create(any()) verify(podOperations, never()).delete() @@ -346,6 +491,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // make sure all are requested. podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 4, rp -> 3)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 4) verify(podOperations).create(podWithAttachedContainerForId(4, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(5, defaultProfile.id)) verify(podOperations).create(podWithAttachedContainerForId(6, defaultProfile.id)) @@ -356,6 +502,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(pendingExecutor(5, defaultProfile.id)) snapshotsStore.updatePod(pendingExecutor(7, rp.id)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 3) verify(podOperations, times(7)).create(any()) verify(podOperations, never()).delete() @@ -364,6 +511,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { waitForExecutorPodsClock.advance(executorIdleTimeout * 2) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 1, rp -> 1)) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) verify(podOperations, times(7)).create(any()) verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "5", "6") verify(podOperations).withLabelIn(SPARK_EXECUTOR_ID_LABEL, "7") @@ -379,6 +527,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.updatePod(deletedExecutor(7)) snapshotsStore.removeDeletedExecutors() snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 0) assert(!podsAllocatorUnderTest.isDeleted("5")) assert(!podsAllocatorUnderTest.isDeleted("6")) assert(!podsAllocatorUnderTest.isDeleted("7")) @@ -399,6 +548,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { .thenReturn(podOperations) podsAllocatorUnderTest.setTotalExpectedExecutors(Map(defaultProfile -> 6)) + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 5) // Initial request of pods verify(podOperations).create(podWithAttachedContainerForId(1)) verify(podOperations).create(podWithAttachedContainerForId(2)) @@ -414,6 +564,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { // We move forward one allocation cycle waitForExecutorPodsClock.setTime(podAllocationDelay + 1) snapshotsStore.notifySubscribers() + assert(podsAllocatorUnderTest.numOutstandingPods.get() == 2) // We request pod 6 verify(podOperations).create(podWithAttachedContainerForId(6)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 861d41cc50ac5..3573ffc07d2c8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -112,6 +112,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) when(kubernetesClient.configMaps()).thenReturn(configMapsOperations) + when(podAllocator.driverPod).thenReturn(None) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, sc, @@ -127,7 +128,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Start all components") { schedulerBackendUnderTest.start() verify(podAllocator).setTotalExpectedExecutors(Map(defaultProfile -> 3)) - verify(podAllocator).start(TEST_SPARK_APP_ID) + verify(podAllocator).start(TEST_SPARK_APP_ID, schedulerBackendUnderTest) verify(lifecycleEventHandler).start(schedulerBackendUnderTest) verify(watchEvents).start(TEST_SPARK_APP_ID) verify(pollEvents).start(TEST_SPARK_APP_ID) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile index f63f2d0d58e22..2dd4d8c4d1e95 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile @@ -27,8 +27,9 @@ RUN mkdir ${SPARK_HOME}/R # Install R 3.6.3 (http://cloud.r-project.org/bin/linux/debian/) RUN \ - echo "deb http://cloud.r-project.org/bin/linux/debian buster-cran35/" >> /etc/apt/sources.list && \ + apt-get update && \ apt install -y gnupg && \ + echo "deb http://cloud.r-project.org/bin/linux/debian buster-cran35/" >> /etc/apt/sources.list && \ (apt-key adv --keyserver keys.gnupg.net --recv-key 'E19F5F87128899B192B1A2C2AD5F960A256A04AF' || apt-key adv --keyserver keys.openpgp.org --recv-key 'E19F5F87128899B192B1A2C2AD5F960A256A04AF') && \ apt-get update && \ apt install -y -t buster-cran35 r-base r-base-dev && \ diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 000362683016b..9e8963d66d964 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 693eea4690866..946416e32cf56 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 0cd5972f97570..b2c635ca78543 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 20f5339c46fef..2542b45fe2ae2 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -28,7 +28,8 @@ import scala.concurrent.duration._ import com.google.common.io.Files import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.scalatest.BeforeAndAfterAll +import org.scalactic.source.Position +import org.scalatest.{BeforeAndAfterAll, Tag} import org.scalatest.concurrent.Eventually._ import org.scalatest.matchers.must.Matchers @@ -41,6 +42,7 @@ import org.apache.spark.util.Utils abstract class BaseYarnClusterSuite extends SparkFunSuite with BeforeAndAfterAll with Matchers with Logging { + private var isBindSuccessful = true // log4j configuration for the YARN containers, so that their output is collected // by YARN instead of trying to overwrite unit-tests.log. @@ -64,6 +66,14 @@ abstract class BaseYarnClusterSuite def newYarnConfig(): YarnConfiguration + override protected def test(testName: String, testTags: Tag*)(testFun: => Any) + (implicit pos: Position): Unit = { + super.test(testName, testTags: _*) { + assume(isBindSuccessful, "Mini Yarn cluster should be able to bind.") + testFun + } + } + override def beforeAll(): Unit = { super.beforeAll() @@ -80,9 +90,16 @@ abstract class BaseYarnClusterSuite yarnConf.set("yarn.nodemanager.disk-health-checker.max-disk-utilization-per-disk-percentage", "100.0") - yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) - yarnCluster.init(yarnConf) - yarnCluster.start() + try { + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + yarnCluster.init(yarnConf) + yarnCluster.start() + } catch { + case e: Throwable if org.apache.commons.lang3.exception.ExceptionUtils.indexOfThrowable( + e, classOf[java.net.BindException]) != -1 => + isBindSuccessful = false + return + } // There's a race in MiniYARNCluster in which start() may return before the RM has updated // its address in the configuration. You can see this in the logs by noticing that when @@ -118,7 +135,7 @@ abstract class BaseYarnClusterSuite override def afterAll(): Unit = { try { - yarnCluster.stop() + if (yarnCluster != null) yarnCluster.stop() } finally { super.afterAll() } diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0728ddbffb3fd..08bb504da9c87 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml 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 e9e8ba842a36f..600a5afe62938 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 @@ -872,16 +872,16 @@ class Analyzer(override val catalogManager: CatalogManager) object ResolveTempViews extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { case u @ UnresolvedRelation(ident, _, isStreaming) => - lookupTempView(ident, isStreaming).getOrElse(u) + lookupAndResolveTempView(ident, isStreaming).getOrElse(u) case i @ InsertIntoStatement(UnresolvedRelation(ident, _, false), _, _, _, _, _) => - lookupTempView(ident) + lookupAndResolveTempView(ident) .map(view => i.copy(table = view)) .getOrElse(i) // TODO (SPARK-27484): handle streaming write commands when we have them. case write: V2WriteCommand => write.table match { case UnresolvedRelation(ident, _, false) => - lookupTempView(ident).map(EliminateSubqueryAliases(_)).map { + lookupAndResolveTempView(ident).map(EliminateSubqueryAliases(_)).map { case r: DataSourceV2Relation => write.withNewTable(r) case _ => throw new AnalysisException("Cannot write into temp view " + s"${ident.quoted} as it's not a data source v2 relation.") @@ -905,8 +905,9 @@ class Analyzer(override val catalogManager: CatalogManager) .getOrElse(u) } - def lookupTempView( - identifier: Seq[String], isStreaming: Boolean = false): Option[LogicalPlan] = { + private def lookupTempView( + identifier: Seq[String], + isStreaming: Boolean = false): Option[LogicalPlan] = { // Permanent View can't refer to temp views, no need to lookup at all. if (isResolvingView && !referredTempViewNames.contains(identifier)) return None @@ -920,7 +921,13 @@ class Analyzer(override val catalogManager: CatalogManager) throw new AnalysisException(s"${identifier.quoted} is not a temp view of streaming " + s"logical plan, please use batch API such as `DataFrameReader.table` to read it.") } - tmpView.map(ResolveRelations.resolveViews) + tmpView + } + + private def lookupAndResolveTempView( + identifier: Seq[String], + isStreaming: Boolean = false): Option[LogicalPlan] = { + lookupTempView(identifier, isStreaming).map(ResolveRelations.resolveViews) } } @@ -1093,6 +1100,9 @@ class Analyzer(override val catalogManager: CatalogManager) executeSameContext(child) } } + // Fail the analysis eagerly because outside AnalysisContext, the unresolved operators + // inside a view maybe resolved incorrectly. + checkAnalysis(newChild) view.copy(child = newChild) case p @ SubqueryAlias(_, view: View) => p.copy(child = resolveViews(view)) @@ -1293,7 +1303,11 @@ class Analyzer(override val catalogManager: CatalogManager) relation.output.flatMap { col => outputNameToStaticName.get(col.name).flatMap(staticPartitions.get) match { case Some(staticValue) => - Some(Alias(Cast(Literal(staticValue), col.dataType), col.name)()) + // SPARK-30844: try our best to follow StoreAssignmentPolicy for static partition + // values but not completely follow because we can't do static type checking due to + // the reason that the parser has erased the type info of static partition values + // and converted them to string. + Some(Alias(AnsiCast(Literal(staticValue), col.dataType), col.name)()) case _ if queryColumns.hasNext => Some(queryColumns.next) case _ => @@ -1321,7 +1335,9 @@ class Analyzer(override val catalogManager: CatalogManager) // ResolveOutputRelation runs, using the query's column names that will match the // table names at that point. because resolution happens after a future rule, create // an UnresolvedAttribute. - EqualNullSafe(UnresolvedAttribute(attr.name), Cast(Literal(value), attr.dataType)) + EqualNullSafe( + UnresolvedAttribute.quoted(attr.name), + Cast(Literal(value), attr.dataType)) case None => throw QueryCompilationErrors.unknownStaticPartitionColError(name) } @@ -3903,22 +3919,32 @@ object UpdateOuterReferences extends Rule[LogicalPlan] { */ object ApplyCharTypePadding extends Rule[LogicalPlan] { + object AttrOrOuterRef { + def unapply(e: Expression): Option[Attribute] = e match { + case a: Attribute => Some(a) + case OuterReference(a: Attribute) => Some(a) + case _ => None + } + } + override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsUp { - case operator if operator.resolved => operator.transformExpressionsUp { + case operator => operator.transformExpressionsUp { + case e if !e.childrenResolved => e + // String literal is treated as char type when it's compared to a char type column. // We should pad the shorter one to the longer length. - case b @ BinaryComparison(attr: Attribute, lit) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + case b @ BinaryComparison(e @ AttrOrOuterRef(attr), lit) if lit.foldable => + padAttrLitCmp(e, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren) }.getOrElse(b) - case b @ BinaryComparison(lit, attr: Attribute) if lit.foldable => - padAttrLitCmp(attr, lit).map { newChildren => + case b @ BinaryComparison(lit, e @ AttrOrOuterRef(attr)) if lit.foldable => + padAttrLitCmp(e, attr.metadata, lit).map { newChildren => b.withNewChildren(newChildren.reverse) }.getOrElse(b) - case i @ In(attr: Attribute, list) + case i @ In(e @ AttrOrOuterRef(attr), list) if attr.dataType == StringType && list.forall(_.foldable) => CharVarcharUtils.getRawType(attr.metadata).flatMap { case CharType(length) => @@ -3927,7 +3953,7 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { val literalCharLengths = literalChars.map(_.numChars()) val targetLen = (length +: literalCharLengths).max Some(i.copy( - value = addPadding(attr, length, targetLen), + value = addPadding(e, length, targetLen), list = list.zip(literalCharLengths).map { case (lit, charLength) => addPadding(lit, charLength, targetLen) } ++ nulls.map(Literal.create(_, StringType)))) @@ -3935,20 +3961,46 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { }.getOrElse(i) // For char type column or inner field comparison, pad the shorter one to the longer length. - case b @ BinaryComparison(left: Attribute, right: Attribute) => - b.withNewChildren(CharVarcharUtils.addPaddingInStringComparison(Seq(left, right))) + case b @ BinaryComparison(e1 @ AttrOrOuterRef(left), e2 @ AttrOrOuterRef(right)) + // For the same attribute, they must be the same length and no padding is needed. + if !left.semanticEquals(right) => + val outerRefs = (e1, e2) match { + case (_: OuterReference, _: OuterReference) => Seq(left, right) + case (_: OuterReference, _) => Seq(left) + case (_, _: OuterReference) => Seq(right) + case _ => Nil + } + val newChildren = CharVarcharUtils.addPaddingInStringComparison(Seq(left, right)) + if (outerRefs.nonEmpty) { + b.withNewChildren(newChildren.map(_.transform { + case a: Attribute if outerRefs.exists(_.semanticEquals(a)) => OuterReference(a) + })) + } else { + b.withNewChildren(newChildren) + } - case i @ In(attr: Attribute, list) if list.forall(_.isInstanceOf[Attribute]) => + case i @ In(e @ AttrOrOuterRef(attr), list) if list.forall(_.isInstanceOf[Attribute]) => val newChildren = CharVarcharUtils.addPaddingInStringComparison( attr +: list.map(_.asInstanceOf[Attribute])) - i.copy(value = newChildren.head, list = newChildren.tail) + if (e.isInstanceOf[OuterReference]) { + i.copy( + value = newChildren.head.transform { + case a: Attribute if a.semanticEquals(attr) => OuterReference(a) + }, + list = newChildren.tail) + } else { + i.copy(value = newChildren.head, list = newChildren.tail) + } } } } - private def padAttrLitCmp(attr: Attribute, lit: Expression): Option[Seq[Expression]] = { - if (attr.dataType == StringType) { - CharVarcharUtils.getRawType(attr.metadata).flatMap { + private def padAttrLitCmp( + expr: Expression, + metadata: Metadata, + lit: Expression): Option[Seq[Expression]] = { + if (expr.dataType == StringType) { + CharVarcharUtils.getRawType(metadata).flatMap { case CharType(length) => val str = lit.eval().asInstanceOf[UTF8String] if (str == null) { @@ -3956,9 +4008,9 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } else { val stringLitLen = str.numChars() if (length < stringLitLen) { - Some(Seq(StringRPad(attr, Literal(stringLitLen)), lit)) + Some(Seq(StringRPad(expr, Literal(stringLitLen)), lit)) } else if (length > stringLitLen) { - Some(Seq(attr, StringRPad(lit, Literal(length)))) + Some(Seq(expr, StringRPad(lit, Literal(length)))) } else { None } @@ -3970,6 +4022,14 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] { } } + private def padOuterRefAttrCmp(outerAttr: Attribute, attr: Attribute): Seq[Expression] = { + val Seq(r, newAttr) = CharVarcharUtils.addPaddingInStringComparison(Seq(outerAttr, attr)) + val newOuterRef = r.transform { + case ar: Attribute if ar.semanticEquals(outerAttr) => OuterReference(ar) + } + Seq(newOuterRef, newAttr) + } + private def addPadding(expr: Expression, charLength: Int, targetLength: Int): Expression = { if (targetLength > charLength) StringRPad(expr, Literal(targetLength)) else expr } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 3e084f0af5121..3dfe7f46d54bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -891,14 +891,72 @@ trait CheckAnalysis extends PredicateHelper { // +- SubqueryAlias t1, `t1` // +- Project [_1#73 AS c1#76, _2#74 AS c2#77] // +- LocalRelation [_1#73, _2#74] - def failOnNonEqualCorrelatedPredicate(found: Boolean, p: LogicalPlan): Unit = { - if (found) { + // SPARK-35080: The same issue can happen to correlated equality predicates when + // they do not guarantee one-to-one mapping between inner and outer attributes. + // For example: + // Table: + // t1(a, b): [(0, 6), (1, 5), (2, 4)] + // t2(c): [(6)] + // + // Query: + // SELECT c, (SELECT COUNT(*) FROM t1 WHERE a + b = c) FROM t2 + // + // Original subquery plan: + // Aggregate [count(1)] + // +- Filter ((a + b) = outer(c)) + // +- LocalRelation [a, b] + // + // Plan after pulling up correlated predicates: + // Aggregate [a, b] [count(1), a, b] + // +- LocalRelation [a, b] + // + // Plan after rewrite: + // Project [c1, count(1)] + // +- Join LeftOuter ((a + b) = c) + // :- LocalRelation [c] + // +- Aggregate [a, b] [count(1), a, b] + // +- LocalRelation [a, b] + // + // The right hand side of the join transformed from the subquery will output + // count(1) | a | b + // 1 | 0 | 6 + // 1 | 1 | 5 + // 1 | 2 | 4 + // and the plan after rewrite will give the original query incorrect results. + def failOnUnsupportedCorrelatedPredicate(predicates: Seq[Expression], p: LogicalPlan): Unit = { + if (predicates.nonEmpty) { // Report a non-supported case as an exception - failAnalysis(s"Correlated column is not allowed in a non-equality predicate:\n$p") + failAnalysis("Correlated column is not allowed in predicate " + + s"${predicates.map(_.sql).mkString}:\n$p") } } - var foundNonEqualCorrelatedPred: Boolean = false + def containsAttribute(e: Expression): Boolean = { + e.find(_.isInstanceOf[Attribute]).isDefined + } + + // Given a correlated predicate, check if it is either a non-equality predicate or + // equality predicate that does not guarantee one-on-one mapping between inner and + // outer attributes. When the correlated predicate does not contain any attribute + // (i.e. only has outer references), it is supported and should return false. E.G.: + // (a = outer(c)) -> false + // (outer(c) = outer(d)) -> false + // (a > outer(c)) -> true + // (a + b = outer(c)) -> true + // The last one is true because there can be multiple combinations of (a, b) that + // satisfy the equality condition. For example, if outer(c) = 0, then both (0, 0) + // and (-1, 1) can make the predicate evaluate to true. + def isUnsupportedPredicate(condition: Expression): Boolean = condition match { + // Only allow equality condition with one side being an attribute and another + // side being an expression without attributes from the inner query. Note + // OuterReference is a leaf node and will not be found here. + case Equality(_: Attribute, b) => containsAttribute(b) + case Equality(a, _: Attribute) => containsAttribute(a) + case e @ Equality(_, _) => containsAttribute(e) + case _ => true + } + + val unsupportedPredicates = mutable.ArrayBuffer.empty[Expression] // Simplify the predicates before validating any unsupported correlation patterns in the plan. AnalysisHelper.allowInvokingTransformsInAnalyzer { BooleanSimplification(sub).foreachUp { @@ -941,22 +999,17 @@ trait CheckAnalysis extends PredicateHelper { // The other operator is Join. Filter can be anywhere in a correlated subquery. case f: Filter => val (correlated, _) = splitConjunctivePredicates(f.condition).partition(containsOuter) - - // Find any non-equality correlated predicates - foundNonEqualCorrelatedPred = foundNonEqualCorrelatedPred || correlated.exists { - case _: EqualTo | _: EqualNullSafe => false - case _ => true - } + unsupportedPredicates ++= correlated.filter(isUnsupportedPredicate) failOnInvalidOuterReference(f) // Aggregate cannot host any correlated expressions // It can be on a correlation path if the correlation contains - // only equality correlated predicates. + // only supported correlated equality predicates. // It cannot be on a correlation path if the correlation has // non-equality correlated predicates. case a: Aggregate => failOnInvalidOuterReference(a) - failOnNonEqualCorrelatedPredicate(foundNonEqualCorrelatedPred, a) + failOnUnsupportedCorrelatedPredicate(unsupportedPredicates.toSeq, a) // Join can host correlated expressions. case j @ Join(left, right, joinType, _, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala index d5c407b47c5be..32bdb82b3b68e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala @@ -100,7 +100,11 @@ object TableOutputResolver { case _ => Cast(queryExpr, tableAttr.dataType, Option(conf.sessionLocalTimeZone)) } - val exprWithStrLenCheck = CharVarcharUtils.stringLengthCheck(casted, tableAttr) + val exprWithStrLenCheck = if (conf.charVarcharAsString) { + casted + } else { + CharVarcharUtils.stringLengthCheck(casted, tableAttr) + } // Renaming is needed for handling the following cases like // 1) Column names/types do not match, e.g., INSERT INTO TABLE tab1 SELECT 1, 2 // 2) Target tables have column metadata diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index efc9e971df72a..55eca6304dfcb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -190,7 +190,7 @@ object UnresolvedAttribute { * Used to split attribute name by dot with backticks rule. * Backticks must appear in pairs, and the quoted string must be a complete name part, * which means `ab..c`e.f is not allowed. - * Escape character is not supported now, so we can't use backtick inside name part. + * We can use backtick only inside quoted name parts. */ def parseAttributeName(name: String): Seq[String] = { def e = new AnalysisException(s"syntax error in attribute name: $name") @@ -202,8 +202,13 @@ object UnresolvedAttribute { val char = name(i) if (inBacktick) { if (char == '`') { - inBacktick = false - if (i + 1 < name.length && name(i + 1) != '.') throw e + if (i + 1 < name.length && name(i + 1) == '`') { + tmp += '`' + i += 1 + } else { + inBacktick = false + if (i + 1 < name.length && name(i + 1) != '.') throw e + } } else { tmp += char } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index dfadf0a539948..fd2315ce519d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.expressions.Alias +import java.util.Locale + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.catalyst.rules.Rule @@ -57,15 +60,44 @@ object EliminateView extends Rule[LogicalPlan] with CastSupport { // The child has the different output attributes with the View operator. Adds a Project over // the child of the view. case v @ View(desc, _, output, child) if child.resolved && !v.sameOutput(child) => + // Use the stored view query output column names to find the matching attributes. The column + // names may have duplication, e.g. `CREATE VIEW v(x, y) AS SELECT 1 col, 2 col`. We need to + // make sure the that matching attributes have the same number of duplications, and pick the + // corresponding attribute by ordinal. val resolver = conf.resolver val queryColumnNames = desc.viewQueryColumnNames val queryOutput = if (queryColumnNames.nonEmpty) { - // Find the attribute that has the expected attribute name from an attribute list, the names - // are compared using conf.resolver. - // `CheckAnalysis` already guarantees the expected attribute can be found for sure. - desc.viewQueryColumnNames.map { colName => - child.output.find(attr => resolver(attr.name, colName)).get + val normalizeColName: String => String = if (conf.caseSensitiveAnalysis) { + identity + } else { + _.toLowerCase(Locale.ROOT) + } + val nameToCounts = scala.collection.mutable.HashMap.empty[String, Int] + val nameToMatchedCols = scala.collection.mutable.HashMap.empty[String, Seq[Attribute]] + + val outputAttrs = queryColumnNames.map { colName => + val normalized = normalizeColName(colName) + val count = nameToCounts.getOrElse(normalized, 0) + val matchedCols = nameToMatchedCols.getOrElseUpdate( + normalized, child.output.filter(attr => resolver(attr.name, colName))) + if (matchedCols.length - 1 < count) { + throw new AnalysisException(s"The SQL query of view ${desc.identifier} has an " + + s"incompatible schema change and column $colName cannot be resolved. Expect " + + s"more attributes named $colName in ${child.output.mkString("[", ",", "]")}") + } + nameToCounts(normalized) = count + 1 + matchedCols(count) } + + nameToCounts.foreach { case (colName, count) => + if (count > 1 && nameToMatchedCols(colName).length != count) { + throw new AnalysisException(s"The SQL query of view ${desc.identifier} has an " + + s"incompatible schema change and column $colName cannot be resolved. Expect " + + s"less attributes named $colName in ${child.output.mkString("[", ",", "]")}") + } + } + + outputAttrs } else { // For view created before Spark 2.2.0, the view text is already fully qualified, the plan // output is the same with the view output. 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 4b132d8ab6c7d..c2b74604715b2 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 @@ -119,12 +119,16 @@ object ExternalCatalogUtils { } } - def getPartitionPathString(col: String, value: String): String = { - val partitionString = if (value == null || value.isEmpty) { + def getPartitionValueString(value: String): String = { + if (value == null || value.isEmpty) { DEFAULT_PARTITION_NAME } else { escapePathName(value) } + } + + def getPartitionPathString(col: String, value: String): String = { + val partitionString = getPartitionValueString(value) escapePathName(col) + "=" + partitionString } 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 90e69469eef69..5809751967cb3 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 @@ -342,8 +342,7 @@ class InMemoryCatalog( } override def tableExists(db: String, table: String): Boolean = synchronized { - requireDbExists(db) - catalog(db).tables.contains(table) + catalog.contains(db) && catalog(db).tables.contains(table) } override def listTables(db: String): Seq[String] = synchronized { @@ -500,7 +499,11 @@ class InMemoryCatalog( newSpec, partitionColumnNames, tablePath) try { val fs = tablePath.getFileSystem(hadoopConfig) - fs.rename(oldPartPath, newPartPath) + fs.mkdirs(newPartPath) + if(!fs.rename(oldPartPath, newPartPath)) { + throw new IOException(s"Renaming partition path from $oldPartPath to " + + s"$newPartPath returned false") + } } catch { case e: IOException => throw new SparkException(s"Unable to rename partition path $oldPartPath", e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index ec405994eadef..b4b25a895016c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -166,8 +166,6 @@ class CSVOptions( val quoteAll = getBool("quoteAll", false) - val inputBufferSize = 128 - /** * The max error content length in CSV parser/writer exception message. */ @@ -213,6 +211,9 @@ class CSVOptions( } val lineSeparatorInWrite: Option[String] = lineSeparator + val inputBufferSize: Option[Int] = parameters.get("inputBufferSize").map(_.toInt) + .orElse(SQLConf.get.getConf(SQLConf.CSV_INPUT_BUFFER_SIZE)) + /** * The handling method to be used when unescaped quotes are found in the input. */ @@ -259,7 +260,7 @@ class CSVOptions( settings.setIgnoreLeadingWhitespaces(ignoreLeadingWhiteSpaceInRead) settings.setIgnoreTrailingWhitespaces(ignoreTrailingWhiteSpaceInRead) settings.setReadInputOnSeparateThread(false) - settings.setInputBufferSize(inputBufferSize) + inputBufferSize.foreach(settings.setInputBufferSize) settings.setMaxColumns(maxColumns) settings.setNullValue(nullValue) settings.setEmptyValue(emptyValueInRead) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala index 665b2cd1274fd..2d8f02860dd7d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala @@ -48,7 +48,7 @@ object OuterScopes { * useful for inner class defined in REPL. */ def getOuterScope(innerCls: Class[_]): () => AnyRef = { - assert(innerCls.isMemberClass) + assert(Utils.isMemberClass(innerCls)) val outerClassName = innerCls.getDeclaringClass.getName val outer = outerScopes.get(outerClassName) if (outer == null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 72bd9ca4d3d1c..80cd6d6c86312 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -1301,7 +1301,7 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit if (Float.isNaN($c) || Float.isInfinite($c)) { $evNull = true; } else { - $evPrim = (long)($c * $MICROS_PER_SECOND); + $evPrim = (long)((double)$c * $MICROS_PER_SECOND); } """ } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala index 241c761624b76..03b5517f6df05 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala @@ -41,9 +41,14 @@ case class ProjectionOverSchema(schema: StructType) { case a: GetArrayStructFields => getProjection(a.child).map(p => (p, p.dataType)).map { case (projection, ArrayType(projSchema @ StructType(_), _)) => + // For case-sensitivity aware field resolution, we should take `ordinal` which + // points to correct struct field. + val selectedField = a.child.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType](a.ordinal) + val prunedField = projSchema(selectedField.name) GetArrayStructFields(projection, - projSchema(a.field.name), - projSchema.fieldIndex(a.field.name), + prunedField.copy(name = a.field.name), + projSchema.fieldIndex(selectedField.name), projSchema.size, a.containsNull) case (_, projSchema) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala index 6213267c41c64..30093ef085913 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala @@ -17,26 +17,30 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.types._ -object SchemaPruning { +object SchemaPruning extends SQLConfHelper { /** - * Filters the schema by the requested fields. For example, if the schema is struct, - * and given requested field are "a", the field "b" is pruned in the returned schema. - * Note that schema field ordering at original schema is still preserved in pruned schema. + * Prunes the nested schema by the requested fields. For example, if the schema is: + * `id int, s struct`, and given requested field "s.a", the inner field "b" + * is pruned in the returned schema: `id int, s struct`. + * Note that: + * 1. The schema field ordering at original schema is still preserved in pruned schema. + * 2. The top-level fields are not pruned here. */ def pruneDataSchema( dataSchema: StructType, requestedRootFields: Seq[RootField]): StructType = { + val resolver = conf.resolver // Merge the requested root fields into a single schema. Note the ordering of the fields // in the resulting schema may differ from their ordering in the logical relation's // original schema val mergedSchema = requestedRootFields - .map { case root: RootField => StructType(Array(root.field)) } + .map { root: RootField => StructType(Array(root.field)) } .reduceLeft(_ merge _) - val dataSchemaFieldNames = dataSchema.fieldNames.toSet val mergedDataSchema = - StructType(mergedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) + StructType(dataSchema.map(d => mergedSchema.find(m => resolver(m.name, d.name)).getOrElse(d))) // Sort the fields of mergedDataSchema according to their order in dataSchema, // recursively. This makes mergedDataSchema a pruned schema of dataSchema sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType] @@ -61,12 +65,15 @@ object SchemaPruning { sortLeftFieldsByRight(leftValueType, rightValueType), containsNull) case (leftStruct: StructType, rightStruct: StructType) => - val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains) + val resolver = conf.resolver + val filteredRightFieldNames = rightStruct.fieldNames + .filter(name => leftStruct.fieldNames.exists(resolver(_, name))) val sortedLeftFields = filteredRightFieldNames.map { fieldName => - val leftFieldType = leftStruct(fieldName).dataType + val resolvedLeftStruct = leftStruct.find(p => resolver(p.name, fieldName)).get + val leftFieldType = resolvedLeftStruct.dataType val rightFieldType = rightStruct(fieldName).dataType val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType) - StructField(fieldName, sortedLeftFieldType, nullable = leftStruct(fieldName).nullable) + StructField(fieldName, sortedLeftFieldType, nullable = resolvedLeftStruct.nullable) } StructType(sortedLeftFields) case _ => left diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala index f2acb75ea6ac4..39dfdf99a9a2b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala @@ -75,7 +75,11 @@ object SelectedField { val field = c.childSchema(c.ordinal) val newField = field.copy(dataType = dataTypeOpt.getOrElse(field.dataType)) selectField(c.child, Option(struct(newField))) - case GetArrayStructFields(child, field, _, _, containsNull) => + case GetArrayStructFields(child, _, ordinal, _, containsNull) => + // For case-sensitivity aware field resolution, we should take `ordinal` which + // points to correct struct field. + val field = child.dataType.asInstanceOf[ArrayType] + .elementType.asInstanceOf[StructType](ordinal) val newFieldDataType = dataTypeOpt match { case None => // GetArrayStructFields is the top level extractor. This means its result is diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala index 8b51e0a908f42..45d55a085a717 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription} +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, ExpressionDescription, Literal} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.sketch.CountMinSketch @@ -142,6 +142,9 @@ case class CountMinSketchAgg( override def dataType: DataType = BinaryType + override def defaultResult: Option[Literal] = + Option(Literal.create(eval(createAggregationBuffer()), dataType)) + override def children: Seq[Expression] = Seq(child, epsExpression, confidenceExpression, seedExpression) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index b3cc9a31fec2f..b6a4d116b16b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -90,6 +90,8 @@ case class HyperLogLogPlusPlus( override def aggBufferSchema: StructType = StructType.fromAttributes(aggBufferAttributes) + override def defaultResult: Option[Literal] = Option(Literal.create(0L, dataType)) + val hllppHelper = new HyperLogLogPlusPlusHelper(relativeSD) /** Allocate enough words to store all registers. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index f95f44c808092..f1b9630312d55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -47,6 +47,8 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper // actual order of input rows. override lazy val deterministic: Boolean = false + override def defaultResult: Option[Literal] = Option(Literal.create(Array(), dataType)) + protected def convertToBufferElement(value: Any): Any override def update(buffer: T, input: InternalRow): T = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 421b8ee2a25b2..9dd370445ec46 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -202,8 +202,7 @@ abstract class AggregateFunction extends Expression { def inputAggBufferAttributes: Seq[AttributeReference] /** - * Result of the aggregate function when the input is empty. This is currently only used for the - * proper rewriting of distinct aggregate functions. + * Result of the aggregate function when the input is empty. */ def defaultResult: Option[Literal] = None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 1ff4a93cf0acd..6e6b9461f3674 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -1089,7 +1089,8 @@ class CodegenContext extends Logging { // Generate the code for this expression tree and wrap it in a function. val fnName = freshName("subExpr") val inputVars = inputVarsForAllFuncs(i) - val argList = inputVars.map(v => s"${v.javaType.getName} ${v.variableName}") + val argList = + inputVars.map(v => s"${CodeGenerator.typeName(v.javaType)} ${v.variableName}") val returnType = javaType(expr.dataType) val fn = s""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index cb59fbda2b3b9..1779d413e025d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCheckResult, TypeCoercion, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.{FUNC_ALIAS, FunctionBuilder} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ @@ -336,6 +336,14 @@ object CreateStruct { */ def apply(children: Seq[Expression]): CreateNamedStruct = { CreateNamedStruct(children.zipWithIndex.flatMap { + // For multi-part column name like `struct(a.b.c)`, it may be resolved into: + // 1. Attribute if `a.b.c` is simply a qualified column name. + // 2. GetStructField if `a.b` refers to a struct-type column. + // 3. GetArrayStructFields if `a.b` refers to a array-of-struct-type column. + // 4. GetMapValue if `a.b` refers to a map-type column. + // We should always use the last part of the column name (`c` in the above example) as the + // alias name inside CreateNamedStruct. + case (u: UnresolvedAttribute, _) => Seq(Literal(u.nameParts.last), u) case (e: NamedExpression, _) if e.resolved => Seq(Literal(e.name), e) case (e: NamedExpression, _) => Seq(NamePlaceholder, e) case (e, index) => Seq(Literal(s"col${index + 1}"), e) 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 c20dd6148be3e..4a27b2a482728 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 @@ -502,7 +502,7 @@ case class SecondsToTimestamp(child: Expression) extends UnaryExpression input.asInstanceOf[Decimal].toJavaBigDecimal.multiply(operand).longValueExact() case _: FloatType => input => val f = input.asInstanceOf[Float] - if (f.isNaN || f.isInfinite) null else (f * MICROS_PER_SECOND).toLong + if (f.isNaN || f.isInfinite) null else (f.toDouble * MICROS_PER_SECOND).toLong case _: DoubleType => input => val d = input.asInstanceOf[Double] if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong @@ -517,13 +517,14 @@ case class SecondsToTimestamp(child: Expression) extends UnaryExpression val operand = s"new java.math.BigDecimal($MICROS_PER_SECOND)" defineCodeGen(ctx, ev, c => s"$c.toJavaBigDecimal().multiply($operand).longValueExact()") case other => + val castToDouble = if (other.isInstanceOf[FloatType]) "(double)" else "" nullSafeCodeGen(ctx, ev, c => { val typeStr = CodeGenerator.boxedType(other) s""" |if ($typeStr.isNaN($c) || $typeStr.isInfinite($c)) { | ${ev.isNull} = true; |} else { - | ${ev.value} = (long)($c * $MICROS_PER_SECOND); + | ${ev.value} = (long)($castToDouble$c * $MICROS_PER_SECOND); |} |""".stripMargin }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 4454afb6c099b..a4e069d652b43 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.Comparator -import java.util.concurrent.atomic.AtomicReference +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import scala.collection.mutable @@ -52,6 +52,16 @@ case class UnresolvedNamedLambdaVariable(nameParts: Seq[String]) override def sql: String = name } +object UnresolvedNamedLambdaVariable { + + // Counter to ensure lambda variable names are unique + private val nextVarNameId = new AtomicInteger(0) + + def freshVarName(name: String): String = { + s"${name}_${nextVarNameId.getAndIncrement()}" + } +} + /** * A named lambda variable. */ @@ -277,7 +287,8 @@ case class ArrayTransform( if (indexVar.isDefined) { indexVar.get.value.set(i) } - result.update(i, f.eval(inputRow)) + val v = InternalRow.copyValue(f.eval(inputRow)) + result.update(i, v) i += 1 } result @@ -796,7 +807,7 @@ case class TransformKeys( while (i < map.numElements) { keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - val result = functionForEval.eval(inputRow) + val result = InternalRow.copyValue(functionForEval.eval(inputRow)) resultKeys.update(i, result) i += 1 } @@ -843,7 +854,8 @@ case class TransformValues( while (i < map.numElements) { keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - resultValues.update(i, functionForEval.eval(inputRow)) + val v = InternalRow.copyValue(functionForEval.eval(inputRow)) + resultValues.update(i, v) i += 1 } new ArrayBasedMapData(map.keyArray(), resultValues) @@ -1026,7 +1038,8 @@ case class MapZipWith(left: Expression, right: Expression, function: Expression) value1Var.value.set(v1) value2Var.value.set(v2) keys.update(i, key) - values.update(i, functionForEval.eval(inputRow)) + val v = InternalRow.copyValue(functionForEval.eval(inputRow)) + values.update(i, v) i += 1 } new ArrayBasedMapData(keys, values) @@ -1098,7 +1111,8 @@ case class ZipWith(left: Expression, right: Expression, function: Expression) } else { rightElemVar.value.set(null) } - result.update(i, f.eval(input)) + val v = InternalRow.copyValue(f.eval(input)) + result.update(i, v) i += 1 } result 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 f391b3128cf41..f78e3f5bf7f61 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 @@ -127,7 +127,12 @@ trait InvokeLike extends Expression with NonSQLExpression { // return null if one of arguments is null null } else { - val ret = method.invoke(obj, args: _*) + val ret = try { + method.invoke(obj, args: _*) + } catch { + // Re-throw the original exception. + case e: java.lang.reflect.InvocationTargetException => throw e.getCause + } val boxedClass = ScalaReflection.typeBoxedJavaMapping.get(dataType) if (boxedClass.isDefined) { boxedClass.get.cast(ret) @@ -136,6 +141,34 @@ trait InvokeLike extends Expression with NonSQLExpression { } } } + + final def findMethod(cls: Class[_], functionName: String, argClasses: Seq[Class[_]]): Method = { + // Looking with function name + argument classes first. + try { + cls.getMethod(functionName, argClasses: _*) + } catch { + case _: NoSuchMethodException => + // For some cases, e.g. arg class is Object, `getMethod` cannot find the method. + // We look at function name + argument length + val m = cls.getMethods.filter { m => + m.getName == functionName && m.getParameterCount == arguments.length + } + if (m.isEmpty) { + sys.error(s"Couldn't find $functionName on $cls") + } else if (m.length > 1) { + // More than one matched method signature. Exclude synthetic one, e.g. generic one. + val realMethods = m.filter(!_.isSynthetic) + if (realMethods.length > 1) { + // Ambiguous case, we don't know which method to choose, just fail it. + sys.error(s"Found ${realMethods.length} $functionName on $cls") + } else { + realMethods.head + } + } else { + m.head + } + } + } } /** @@ -227,7 +260,7 @@ case class StaticInvoke( override def children: Seq[Expression] = arguments lazy val argClasses = ScalaReflection.expressionJavaClasses(arguments) - @transient lazy val method = cls.getDeclaredMethod(functionName, argClasses : _*) + @transient lazy val method = findMethod(cls, functionName, argClasses) override def eval(input: InternalRow): Any = { invoke(null, method, arguments, input, dataType) @@ -314,12 +347,7 @@ case class Invoke( @transient lazy val method = targetObject.dataType match { case ObjectType(cls) => - val m = cls.getMethods.find(_.getName == encodedFunctionName) - if (m.isEmpty) { - sys.error(s"Couldn't find $encodedFunctionName on $cls") - } else { - m - } + Some(findMethod(cls, encodedFunctionName, argClasses)) case _ => None } @@ -332,7 +360,7 @@ case class Invoke( val invokeMethod = if (method.isDefined) { method.get } else { - obj.getClass.getDeclaredMethod(functionName, argClasses: _*) + obj.getClass.getMethod(functionName, argClasses: _*) } invoke(obj, invokeMethod, arguments, input, dataType) } @@ -443,7 +471,7 @@ case class NewInstance( // Note that static inner classes (e.g., inner classes within Scala objects) don't need // outer pointer registration. val needOuterPointer = - outerPointer.isEmpty && cls.isMemberClass && !Modifier.isStatic(cls.getModifiers) + outerPointer.isEmpty && Utils.isMemberClass(cls) && !Modifier.isStatic(cls.getModifiers) childrenResolved && !needOuterPointer } @@ -456,7 +484,6 @@ case class NewInstance( } outerPointer.map { p => val outerObj = p() - val d = outerObj.getClass +: paramTypes val c = getConstructor(outerObj.getClass +: paramTypes) (args: Seq[AnyRef]) => { c(outerObj +: args) @@ -489,7 +516,7 @@ case class NewInstance( // that might be defined on the companion object. case 0 => s"$className$$.MODULE$$.apply($argString)" case _ => outer.map { gen => - s"${gen.value}.new ${cls.getSimpleName}($argString)" + s"${gen.value}.new ${Utils.getSimpleName(cls)}($argString)" }.getOrElse { s"new $className($argString)" } 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 c41686da79487..828f768f17701 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 @@ -348,12 +348,30 @@ object JoinReorderDP extends PredicateHelper with Logging { } } + /** + * To identify the plan with smaller computational cost, + * we use the weighted geometric mean of ratio of rows and the ratio of sizes in bytes. + * + * There are other ways to combine these values as a cost comparison function. + * Some of these, that we have experimented with, but have gotten worse result, + * than with the current one: + * 1) Weighted arithmetic mean of these two ratios - adding up fractions puts + * less emphasis on ratios between 0 and 1. Ratios 10 and 0.1 should be considered + * to be just as strong evidences in opposite directions. The arithmetic mean of these + * would be heavily biased towards the 10. + * 2) Absolute cost (cost = weight * rowCount + (1 - weight) * size) - when adding up + * two numeric measurements that have different units we can easily end up with one + * overwhelming the other. + */ def betterThan(other: JoinPlan, conf: SQLConf): Boolean = { - 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 + 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) + Math.pow(relativeRows.doubleValue, conf.joinReorderCardWeight) * + Math.pow(relativeSize.doubleValue, 1 - conf.joinReorderCardWeight) < 1 + } } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala index 465d2efe2775c..d3127b5560178 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/UpdateFields.scala @@ -50,28 +50,22 @@ object OptimizeUpdateFields extends Rule[LogicalPlan] { val values = withFields.map(_.valExpr) val newNames = mutable.ArrayBuffer.empty[String] - val newValues = mutable.ArrayBuffer.empty[Expression] + val newValues = mutable.HashMap.empty[String, Expression] + // Used to remember the casing of the last instance + val nameMap = mutable.HashMap.empty[String, String] - if (caseSensitive) { - names.zip(values).reverse.foreach { case (name, value) => - if (!newNames.contains(name)) { - newNames += name - newValues += value - } - } - } else { - val nameSet = mutable.HashSet.empty[String] - names.zip(values).reverse.foreach { case (name, value) => - val lowercaseName = name.toLowerCase(Locale.ROOT) - if (!nameSet.contains(lowercaseName)) { - newNames += name - newValues += value - nameSet += lowercaseName - } + names.zip(values).foreach { case (name, value) => + val normalizedName = if (caseSensitive) name else name.toLowerCase(Locale.ROOT) + if (nameMap.contains(normalizedName)) { + newValues += normalizedName -> value + } else { + newNames += normalizedName + newValues += normalizedName -> value } + nameMap += normalizedName -> name } - val newWithFields = newNames.reverse.zip(newValues.reverse).map(p => WithField(p._1, p._2)) + val newWithFields = newNames.map(n => WithField(nameMap(n), newValues(n))) UpdateFields(structExpr, newWithFields.toSeq) case UpdateFields(UpdateFields(struct, fieldOps1), fieldOps2) => 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 78d7cccc09dfb..d9897530ff74c 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 @@ -576,7 +576,9 @@ object LikeSimplification extends Rule[LogicalPlan] { 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) + patterns.map { p => + p -> Option(p).flatMap(p => simplifyLike(child, p.toString)) + }.partition(_._2.isEmpty) val remainPatterns = remainPatternMap.map(_._1) val replacements = replacementMap.map(_._2.get) if (replacements.isEmpty) { 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 c7ca4b5ca6fda..7ace11efe52fb 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 @@ -491,7 +491,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with SQLConfHelper with Logg // Before calling `toMap`, we check duplicated keys to avoid silently ignore partition values // in partition spec like PARTITION(a='1', b='2', a='3'). The real semantical check for // partition columns will be done in analyzer. - checkDuplicateKeys(parts.toSeq, ctx) + if (conf.caseSensitiveAnalysis) { + checkDuplicateKeys(parts.toSeq, ctx) + } else { + checkDuplicateKeys(parts.map(kv => kv._1.toLowerCase(Locale.ROOT) -> kv._2).toSeq, ctx) + } parts.toMap } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 2880e87ab1566..c22a874779fca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -333,7 +333,7 @@ object PhysicalAggregation { case ue: PythonUDF if PythonUDF.isGroupedAggPandasUDF(ue) => equivalentAggregateExpressions.getEquivalentExprs(ue).headOption .getOrElse(ue).asInstanceOf[PythonUDF].resultAttribute - case expression => + case expression if !expression.foldable => // Since we're using `namedGroupingAttributes` to extract the grouping key // columns, we need to replace grouping key expressions with their corresponding // attributes. We do not rely on the equality check at here since attributes may diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala index 732c8ce2b5d98..89bd865391b5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} /** * A logical node that represents a non-query command to be executed by the system. For example, @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute */ trait Command extends LogicalPlan { override def output: Seq[Attribute] = Seq.empty + override def producedAttributes: AttributeSet = outputSet override def children: Seq[LogicalPlan] = Seq.empty // Commands are eagerly executed. They will be converted to LocalRelation after the DataFrame // is created. That said, the statistics of a command is useless. Here we just return a dummy diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ad5c3fd74e9b5..3ea79b35e8c52 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -33,7 +33,10 @@ abstract class LogicalPlan with QueryPlanConstraints with Logging { - /** Metadata fields that can be projected from this node */ + /** + * Metadata fields that can be projected from this node. + * Should be overridden if the plan does not propagate its children's output. + */ def metadataOutput: Seq[Attribute] = children.flatMap(_.metadataOutput) /** Returns true if this subtree has data from a streaming data source. */ @@ -89,11 +92,14 @@ abstract class LogicalPlan } } - private[this] lazy val childAttributes = - AttributeSeq(children.flatMap(c => c.output ++ c.metadataOutput)) + private[this] lazy val childAttributes = AttributeSeq(children.flatMap(_.output)) + + private[this] lazy val childMetadataAttributes = AttributeSeq(children.flatMap(_.metadataOutput)) private[this] lazy val outputAttributes = AttributeSeq(output) + private[this] lazy val outputMetadataAttributes = AttributeSeq(metadataOutput) + /** * Optionally resolves the given strings to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as @@ -103,6 +109,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = childAttributes.resolve(nameParts, resolver) + .orElse(childMetadataAttributes.resolve(nameParts, resolver)) /** * Optionally resolves the given strings to a [[NamedExpression]] based on the output of this @@ -113,6 +120,7 @@ abstract class LogicalPlan nameParts: Seq[String], resolver: Resolver): Option[NamedExpression] = outputAttributes.resolve(nameParts, resolver) + .orElse(outputMetadataAttributes.resolve(nameParts, resolver)) /** * Given an attribute name, split it to name parts by dot, but @@ -122,7 +130,7 @@ abstract class LogicalPlan def resolveQuoted( name: String, resolver: Resolver): Option[NamedExpression] = { - outputAttributes.resolve(UnresolvedAttribute.parseAttributeName(name), resolver) + resolve(UnresolvedAttribute.parseAttributeName(name), resolver) } /** 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 ab24aa422d738..224e7bceec05b 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 @@ -59,6 +59,7 @@ object Subquery { case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = projectList.map(_.toAttribute) + override def metadataOutput: Seq[Attribute] = Nil override def maxRows: Option[Long] = child.maxRows override lazy val resolved: Boolean = { @@ -185,6 +186,8 @@ case class Intersect( leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) } + override def metadataOutput: Seq[Attribute] = Nil + override protected lazy val validConstraints: ExpressionSet = leftConstraints.union(rightConstraints) @@ -205,6 +208,8 @@ case class Except( /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output + override def metadataOutput: Seq[Attribute] = Nil + override protected lazy val validConstraints: ExpressionSet = leftConstraints } @@ -268,6 +273,8 @@ case class Union( } } + override def metadataOutput: Seq[Attribute] = Nil + override lazy val resolved: Boolean = { // allChildrenCompatible needs to be evaluated after childrenResolved def allChildrenCompatible: Boolean = @@ -343,6 +350,17 @@ case class Join( } } + override def metadataOutput: Seq[Attribute] = { + joinType match { + case ExistenceJoin(_) => + left.metadataOutput + case LeftExistence(_) => + left.metadataOutput + case _ => + children.flatMap(_.metadataOutput) + } + } + override protected lazy val validConstraints: ExpressionSet = { joinType match { case _: InnerLike if condition.isDefined => @@ -419,6 +437,7 @@ case class InsertIntoDir( extends UnaryNode { override def output: Seq[Attribute] = Seq.empty + override def metadataOutput: Seq[Attribute] = Nil override lazy val resolved: Boolean = false } @@ -449,6 +468,8 @@ case class View( override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) + override def metadataOutput: Seq[Attribute] = Nil + override def simpleString(maxFields: Int): String = { s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } @@ -616,6 +637,7 @@ case class Aggregate( } override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) + override def metadataOutput: Seq[Attribute] = Nil override def maxRows: Option[Long] = { if (groupingExpressions.isEmpty) { Some(1L) @@ -639,6 +661,8 @@ case class Window( override def output: Seq[Attribute] = child.output ++ windowExpressions.map(_.toAttribute) + override def producedAttributes: AttributeSet = windowOutputSet + def windowOutputSet: AttributeSet = AttributeSet(windowExpressions.map(_.toAttribute)) } @@ -749,6 +773,8 @@ case class Expand( override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) + override def metadataOutput: Seq[Attribute] = Nil + override def producedAttributes: AttributeSet = AttributeSet(output diff child.output) // This operator can reuse attributes (for example making them null when doing a roll up) so @@ -811,6 +837,7 @@ case class Pivot( } groupByExprsOpt.getOrElse(Seq.empty).map(_.toAttribute) ++ pivotAgg } + override def metadataOutput: Seq[Attribute] = Nil } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 6f7fd88f5214d..2be3cb64acc03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -287,14 +287,18 @@ case class DropNamespace( */ case class DescribeNamespace( namespace: LogicalPlan, - extended: Boolean) extends Command { + extended: Boolean, + override val output: Seq[Attribute] = DescribeNamespace.getOutputAttr) extends Command { override def children: Seq[LogicalPlan] = Seq(namespace) - override def output: Seq[Attribute] = Seq( - AttributeReference("name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("value", StringType, nullable = true, - new MetadataBuilder().putString("comment", "value of the column").build())()) +} + +object DescribeNamespace { + def getOutputAttr: Seq[Attribute] = Seq( + AttributeReference("info_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the namespace info").build())(), + AttributeReference("info_value", StringType, nullable = true, + new MetadataBuilder().putString("comment", "value of the namespace info").build())()) } /** @@ -324,7 +328,6 @@ case class ShowNamespaces( pattern: Option[String], override val output: Seq[Attribute] = ShowNamespaces.OUTPUT) extends Command { override def children: Seq[LogicalPlan] = Seq(namespace) - override def producedAttributes: AttributeSet = outputSet } object ShowNamespaces { @@ -337,9 +340,13 @@ object ShowNamespaces { case class DescribeRelation( relation: LogicalPlan, partitionSpec: TablePartitionSpec, - isExtended: Boolean) extends Command { + isExtended: Boolean, + override val output: Seq[Attribute] = DescribeRelation.getOutputAttrs) extends Command { override def children: Seq[LogicalPlan] = Seq(relation) - override def output: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() +} + +object DescribeRelation { + def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeTableAttributes() } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 75a1f712671ee..d6da04e06788f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.trees import java.util.UUID import scala.collection.{mutable, Map} +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.apache.commons.lang3.ClassUtils @@ -536,6 +537,16 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] + private def redactMapString[K, V](map: Map[K, V], maxFields: Int): List[String] = { + // For security reason, redact the map value if the key is in centain patterns + val redactedMap = SQLConf.get.redactOptions(map.toMap) + // construct the redacted map as strings of the format "key=value" + val keyValuePairs = redactedMap.toSeq.map { item => + item._1 + "=" + item._2 + } + truncatedString(keyValuePairs, "[", ", ", "]", maxFields) :: Nil + } + /** Returns a string representing the arguments to this node, minus any children */ def argString(maxFields: Int): String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil @@ -552,8 +563,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case None => Nil case Some(null) => Nil case Some(any) => any :: Nil - case map: CaseInsensitiveStringMap => truncatedString( - map.asCaseSensitiveMap().entrySet().toArray(), "[", ", ", "]", maxFields) :: Nil + case map: CaseInsensitiveStringMap => + redactMapString(map.asCaseSensitiveMap().asScala, maxFields) + case map: Map[_, _] => + redactMapString(map, maxFields) case table: CatalogTable => table.storage.serde match { case Some(serde) => table.identifier :: serde :: Nil @@ -787,9 +800,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { ("deserialized" -> s.deserialized) ~ ("replication" -> s.replication) case n: TreeNode[_] => n.jsonValue case o: Option[_] => o.map(parseToJson) - // Recursive scan Seq[TreeNode], Seq[Partitioning], Seq[DataType] - case t: Seq[_] if t.forall(_.isInstanceOf[TreeNode[_]]) || - t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => + // Recursive scan Seq[Partitioning], Seq[DataType], Seq[Product] + case t: Seq[_] if t.forall(_.isInstanceOf[Partitioning]) || + t.forall(_.isInstanceOf[DataType]) || + t.forall(_.isInstanceOf[Product]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => JString(truncatedString(t, "[", ", ", "]", SQLConf.get.maxToStringFields)) @@ -827,6 +841,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case broadcast: BroadcastMode => true case table: CatalogTableType => true case storage: CatalogStorageFormat => true + // Write out product that contains TreeNode, since there are some Tuples such as cteRelations + // in With, branches in CaseWhen which are essential to understand the plan. + case p if p.productIterator.exists(_.isInstanceOf[TreeNode[_]]) => true case _ => false } } 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 87cf3c93ba26e..89cb67cc4b773 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 @@ -50,7 +50,10 @@ object DateTimeUtils { val TIMEZONE_OPTION = "timeZone" - def getZoneId(timeZoneId: String): ZoneId = ZoneId.of(timeZoneId, ZoneId.SHORT_IDS) + def getZoneId(timeZoneId: String): ZoneId = { + // To support the (+|-)h:mm format because it was supported before Spark 3.0. + ZoneId.of(timeZoneId.replaceFirst("(\\+|\\-)(\\d):", "$10$2:"), ZoneId.SHORT_IDS) + } def getTimeZone(timeZoneId: String): TimeZone = TimeZone.getTimeZone(getZoneId(timeZoneId)) /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 7dbdd1ef1cdc5..dca75e5083331 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -21,26 +21,6 @@ import org.apache.spark.unsafe.types.UTF8String object NumberConverter { - /** - * Divide x by m as if x is an unsigned 64-bit integer. Examples: - * unsignedLongDiv(-1, 2) == Long.MAX_VALUE unsignedLongDiv(6, 3) == 2 - * unsignedLongDiv(0, 5) == 0 - * - * @param x is treated as unsigned - * @param m is treated as signed - */ - private def unsignedLongDiv(x: Long, m: Int): Long = { - if (x >= 0) { - x / m - } else { - // Let uval be the value of the unsigned long with the same bits as x - // Two's complement => x = uval - 2*MAX - 2 - // => uval = x + 2*MAX + 2 - // Now, use the fact: (a+b)/c = a/c + b/c + (a%c+b%c)/c - x / m + 2 * (Long.MaxValue / m) + 2 / m + (x % m + 2 * (Long.MaxValue % m) + 2 % m) / m - } - } - /** * Decode v into value[]. * @@ -52,7 +32,7 @@ object NumberConverter { java.util.Arrays.fill(value, 0.asInstanceOf[Byte]) var i = value.length - 1 while (tmpV != 0) { - val q = unsignedLongDiv(tmpV, radix) + val q = java.lang.Long.divideUnsigned(tmpV, radix) value(i) = (tmpV - q * radix).asInstanceOf[Byte] tmpV = q i -= 1 @@ -69,12 +49,12 @@ object NumberConverter { */ private def encode(radix: Int, fromPos: Int, value: Array[Byte]): Long = { var v: Long = 0L - val bound = unsignedLongDiv(-1 - radix, radix) // Possible overflow once + val bound = java.lang.Long.divideUnsigned(-1 - radix, radix) // Possible overflow once var i = fromPos while (i < value.length && value(i) >= 0) { if (v >= bound) { // Check for overflow - if (unsignedLongDiv(-1 - value(i), radix) < v) { + if (java.lang.Long.divideUnsigned(-1 - value(i), radix) < v) { return -1 } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 46860ae1771de..2999d475fc8f6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -29,6 +29,7 @@ import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.util.Utils /** * The collection of functions for rebasing days and microseconds from/to the hybrid calendar @@ -263,7 +264,7 @@ object RebaseDateTime { // `JsonRebaseRecord`. AnyRefMap is used here instead of Scala's immutable map because // it is 2 times faster in DateTimeRebaseBenchmark. private[sql] def loadRebaseRecords(fileName: String): AnyRefMap[String, RebaseInfo] = { - val file = Thread.currentThread().getContextClassLoader.getResource(fileName) + val file = Utils.getSparkClassLoader.getResource(fileName) val mapper = new ObjectMapper() with ScalaObjectMapper mapper.registerModule(DefaultScalaModule) val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fcdf91071404f..2fcfba163979f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -474,7 +474,7 @@ object SQLConf { val COALESCE_PARTITIONS_INITIAL_PARTITION_NUM = buildConf("spark.sql.adaptive.coalescePartitions.initialPartitionNum") - .doc("The initial number of shuffle partitions before coalescing. By default it equals to " + + .doc("The initial number of shuffle partitions before coalescing. If not set, it equals to " + s"${SHUFFLE_PARTITIONS.key}. This configuration only has an effect when " + s"'${ADAPTIVE_EXECUTION_ENABLED.key}' and '${COALESCE_PARTITIONS_ENABLED.key}' " + "are both true.") @@ -491,8 +491,8 @@ object SQLConf { "reduce IO and improve performance. Note, multiple contiguous blocks exist in single " + s"fetch request only happen when '${ADAPTIVE_EXECUTION_ENABLED.key}' and " + s"'${COALESCE_PARTITIONS_ENABLED.key}' are both true. This feature also depends " + - "on a relocatable serializer, the concatenation support codec in use and the new version " + - "shuffle fetch protocol.") + "on a relocatable serializer, the concatenation support codec in use, the new version " + + "shuffle fetch protocol and io encryption is disabled.") .version("3.0.0") .booleanConf .createWithDefault(true) @@ -1860,8 +1860,10 @@ object SQLConf { val JOIN_REORDER_CARD_WEIGHT = buildConf("spark.sql.cbo.joinReorder.card.weight") .internal() - .doc("The weight of cardinality (number of rows) for plan cost comparison in join reorder: " + - "rows * weight + size * (1 - weight).") + .doc("The weight of the ratio of cardinalities (number of rows) " + + "in the cost comparison function. The ratio of sizes in bytes has weight " + + "1 - this value. The weighted geometric mean of these ratios is used to decide " + + "which of the candidate plans will be chosen by the CBO.") .version("2.2.0") .doubleConf .checkValue(weight => weight >= 0 && weight <= 1, "The weight value must be in [0, 1].") @@ -2373,6 +2375,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val CSV_INPUT_BUFFER_SIZE = buildConf("spark.sql.csv.parser.inputBufferSize") + .internal() + .doc("If it is set, it configures the buffer size of CSV input during parsing. " + + "It is the same as inputBufferSize option in CSV which has a higher priority. " + + "Note that this is a workaround for the parsing library's regression, and this " + + "configuration is internal and supposed to be removed in the near future.") + .version("3.0.3") + .intConf + .createOptional + val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled") .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " + "displayed if and only if the REPL supports the eager evaluation. Currently, the " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 35ad864db0e7d..d506a1521e183 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.Stable * * Please use the singleton `DataTypes.CalendarIntervalType` to refer the type. * - * @note Calendar intervals support comparison and ordering since 3.0.0. + * @note Calendar intervals are not comparable. * * @since 1.5.0 */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala index 004725487911a..cf30c713d8d0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/PartitioningUtils.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME import org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{CharType, StructType, VarcharType} import org.apache.spark.unsafe.types.UTF8String @@ -44,23 +45,24 @@ private[sql] object PartitioningUtils { throw new AnalysisException(s"$key is not a valid partition column in table $tblName.") } - val normalizedVal = normalizedFiled.dataType match { - case CharType(len) if value != null && value != DEFAULT_PARTITION_NAME => - val v = value match { - case Some(str: String) => Some(charTypeWriteSideCheck(str, len)) - case str: String => charTypeWriteSideCheck(str, len) - case other => other - } - v.asInstanceOf[T] - case VarcharType(len) if value != null && value != DEFAULT_PARTITION_NAME => - val v = value match { - case Some(str: String) => Some(varcharTypeWriteSideCheck(str, len)) - case str: String => varcharTypeWriteSideCheck(str, len) - case other => other - } - v.asInstanceOf[T] - case _ => value - } + val normalizedVal = + if (SQLConf.get.charVarcharAsString) value else normalizedFiled.dataType match { + case CharType(len) if value != null && value != DEFAULT_PARTITION_NAME => + val v = value match { + case Some(str: String) => Some(charTypeWriteSideCheck(str, len)) + case str: String => charTypeWriteSideCheck(str, len) + case other => other + } + v.asInstanceOf[T] + case VarcharType(len) if value != null && value != DEFAULT_PARTITION_NAME => + val v = value match { + case Some(str: String) => Some(varcharTypeWriteSideCheck(str, len)) + case str: String => varcharTypeWriteSideCheck(str, len) + case other => other + } + v.asInstanceOf[T] + case _ => value + } normalizedFiled.name -> normalizedVal } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala index 950e313fb727a..352afaa2740ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.types._ * Benchmark `UnsafeProjection` for fixed-length/primitive-type fields. * {{{ * To run this benchmark: - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/UnsafeProjectionBenchmark-results.txt". diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 44128c4419951..20ba9c5f30426 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -700,4 +700,28 @@ class AnalysisErrorSuite extends AnalysisTest { UnresolvedRelation(TableIdentifier("t", Option("nonexist"))))))) assertAnalysisError(plan, "Table or view not found:" :: Nil) } + + test("SPARK-35080: Unsupported correlated equality predicates in subquery") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val c = AttributeReference("c", IntegerType)() + val t1 = LocalRelation(a, b) + val t2 = LocalRelation(c) + val conditions = Seq( + (abs($"a") === $"c", "abs(`a`) = outer(`c`)"), + (abs($"a") <=> $"c", "abs(`a`) <=> outer(`c`)"), + ($"a" + 1 === $"c", "(`a` + 1) = outer(`c`)"), + ($"a" + $"b" === $"c", "(`a` + `b`) = outer(`c`)"), + ($"a" + $"c" === $"b", "(`a` + outer(`c`)) = `b`"), + (And($"a" === $"c", Cast($"a", IntegerType) === $"c"), "CAST(`a` AS INT) = outer(`c`)")) + conditions.foreach { case (cond, msg) => + val plan = Project( + ScalarSubquery( + Aggregate(Nil, count(Literal(1)).as("cnt") :: Nil, + Filter(cond, t1)) + ).as("sub") :: Nil, + t2) + assertAnalysisError(plan, s"Correlated column is not allowed in predicate ($msg)" :: Nil) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala index 3e9a8b71a8fb6..ec9480514ba2d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TableLookupCacheSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import java.io.File +import scala.collection.JavaConverters._ + import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock @@ -27,8 +29,8 @@ import org.scalatest.matchers.must.Matchers import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.connector.InMemoryTableCatalog -import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table, V1Table} +import org.apache.spark.sql.connector.{InMemoryTable, InMemoryTableCatalog} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogNotFoundException, Identifier, Table} import org.apache.spark.sql.types._ class TableLookupCacheSuite extends AnalysisTest with Matchers { @@ -46,7 +48,12 @@ class TableLookupCacheSuite extends AnalysisTest with Matchers { ignoreIfExists = false) val v2Catalog = new InMemoryTableCatalog { override def loadTable(ident: Identifier): Table = { - V1Table(externalCatalog.getTable("default", ident.name)) + val catalogTable = externalCatalog.getTable("default", ident.name) + new InMemoryTable( + catalogTable.identifier.table, + catalogTable.schema, + Array.empty, + Map.empty[String, String].asJava) } override def name: String = CatalogManager.SESSION_CATALOG_NAME } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 98f9ce6fe9dbb..ad996dbc509be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -691,6 +691,9 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.createTempView("tbl3", tempTable, overrideIfExists = false) // tableExists should not check temp view. assert(!catalog.tableExists(TableIdentifier("tbl3"))) + + // If database doesn't exist, return false instead of failing. + assert(!catalog.tableExists(TableIdentifier("tbl1", Some("non-exist")))) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index f2598a925e08e..7faab4e7aa757 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -195,8 +195,9 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encoderFor(Encoders.javaSerialization[JavaSerializable])) // test product encoders - private def productTest[T <: Product : ExpressionEncoder](input: T): Unit = { - encodeDecodeTest(input, input.getClass.getSimpleName) + private def productTest[T <: Product : ExpressionEncoder]( + input: T, useFallback: Boolean = false): Unit = { + encodeDecodeTest(input, input.getClass.getSimpleName, useFallback) } case class InnerClass(i: Int) @@ -205,6 +206,90 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(Array(Option(InnerClass(1))), "array of optional inner class") + // holder class to trigger Class.getSimpleName issue + object MalformedClassObject extends Serializable { + case class MalformedNameExample(x: Int) + } + + { + OuterScopes.addOuterScope(MalformedClassObject) + encodeDecodeTest( + MalformedClassObject.MalformedNameExample(42), + "nested Scala class should work", + useFallback = true) + } + + object OuterLevelWithVeryVeryVeryLongClassName1 { + object OuterLevelWithVeryVeryVeryLongClassName2 { + object OuterLevelWithVeryVeryVeryLongClassName3 { + object OuterLevelWithVeryVeryVeryLongClassName4 { + object OuterLevelWithVeryVeryVeryLongClassName5 { + object OuterLevelWithVeryVeryVeryLongClassName6 { + object OuterLevelWithVeryVeryVeryLongClassName7 { + object OuterLevelWithVeryVeryVeryLongClassName8 { + object OuterLevelWithVeryVeryVeryLongClassName9 { + object OuterLevelWithVeryVeryVeryLongClassName10 { + object OuterLevelWithVeryVeryVeryLongClassName11 { + object OuterLevelWithVeryVeryVeryLongClassName12 { + object OuterLevelWithVeryVeryVeryLongClassName13 { + object OuterLevelWithVeryVeryVeryLongClassName14 { + object OuterLevelWithVeryVeryVeryLongClassName15 { + object OuterLevelWithVeryVeryVeryLongClassName16 { + object OuterLevelWithVeryVeryVeryLongClassName17 { + object OuterLevelWithVeryVeryVeryLongClassName18 { + object OuterLevelWithVeryVeryVeryLongClassName19 { + object OuterLevelWithVeryVeryVeryLongClassName20 { + case class MalformedNameExample(x: Int) + }}}}}}}}}}}}}}}}}}}} + + { + OuterScopes.addOuterScope( + OuterLevelWithVeryVeryVeryLongClassName1 + .OuterLevelWithVeryVeryVeryLongClassName2 + .OuterLevelWithVeryVeryVeryLongClassName3 + .OuterLevelWithVeryVeryVeryLongClassName4 + .OuterLevelWithVeryVeryVeryLongClassName5 + .OuterLevelWithVeryVeryVeryLongClassName6 + .OuterLevelWithVeryVeryVeryLongClassName7 + .OuterLevelWithVeryVeryVeryLongClassName8 + .OuterLevelWithVeryVeryVeryLongClassName9 + .OuterLevelWithVeryVeryVeryLongClassName10 + .OuterLevelWithVeryVeryVeryLongClassName11 + .OuterLevelWithVeryVeryVeryLongClassName12 + .OuterLevelWithVeryVeryVeryLongClassName13 + .OuterLevelWithVeryVeryVeryLongClassName14 + .OuterLevelWithVeryVeryVeryLongClassName15 + .OuterLevelWithVeryVeryVeryLongClassName16 + .OuterLevelWithVeryVeryVeryLongClassName17 + .OuterLevelWithVeryVeryVeryLongClassName18 + .OuterLevelWithVeryVeryVeryLongClassName19 + .OuterLevelWithVeryVeryVeryLongClassName20) + encodeDecodeTest( + OuterLevelWithVeryVeryVeryLongClassName1 + .OuterLevelWithVeryVeryVeryLongClassName2 + .OuterLevelWithVeryVeryVeryLongClassName3 + .OuterLevelWithVeryVeryVeryLongClassName4 + .OuterLevelWithVeryVeryVeryLongClassName5 + .OuterLevelWithVeryVeryVeryLongClassName6 + .OuterLevelWithVeryVeryVeryLongClassName7 + .OuterLevelWithVeryVeryVeryLongClassName8 + .OuterLevelWithVeryVeryVeryLongClassName9 + .OuterLevelWithVeryVeryVeryLongClassName10 + .OuterLevelWithVeryVeryVeryLongClassName11 + .OuterLevelWithVeryVeryVeryLongClassName12 + .OuterLevelWithVeryVeryVeryLongClassName13 + .OuterLevelWithVeryVeryVeryLongClassName14 + .OuterLevelWithVeryVeryVeryLongClassName15 + .OuterLevelWithVeryVeryVeryLongClassName16 + .OuterLevelWithVeryVeryVeryLongClassName17 + .OuterLevelWithVeryVeryVeryLongClassName18 + .OuterLevelWithVeryVeryVeryLongClassName19 + .OuterLevelWithVeryVeryVeryLongClassName20 + .MalformedNameExample(42), + "deeply nested Scala class should work", + useFallback = true) + } + productTest(PrimitiveData(1, 1, 1, 1, 1, 1, true)) productTest( @@ -214,7 +299,8 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest(OptionalData(None, None, None, None, None, None, None, None, None)) encodeDecodeTest(Seq(Some(1), None), "Option in array") - encodeDecodeTest(Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), "Option in map") + encodeDecodeTest(Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), "Option in map", + useFallback = true) productTest(BoxedData(1, 1L, 1.0, 1.0f, 1.toShort, 1.toByte, true)) @@ -232,7 +318,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes Map(1 -> null), PrimitiveData(1, 1, 1, 1, 1, 1, true))) - productTest(NestedArray(Array(Array(1, -2, 3), null, Array(4, 5, -6)))) + productTest(NestedArray(Array(Array(1, -2, 3), null, Array(4, 5, -6))), useFallback = true) productTest(("Seq[(String, String)]", Seq(("a", "b")))) @@ -392,8 +478,10 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest((1, FooEnum.E1), "Tuple with Int and scala Enum") encodeDecodeTest((null, FooEnum.E1, FooEnum.E2), "Tuple with Null and scala Enum") encodeDecodeTest(Seq(FooEnum.E1, null), "Seq with scala Enum") - encodeDecodeTest(Map("key" -> FooEnum.E1), "Map with String key and scala Enum") - encodeDecodeTest(Map(FooEnum.E1 -> "value"), "Map with scala Enum key and String value") + encodeDecodeTest(Map("key" -> FooEnum.E1), "Map with String key and scala Enum", + useFallback = true) + encodeDecodeTest(Map(FooEnum.E1 -> "value"), "Map with scala Enum key and String value", + useFallback = true) encodeDecodeTest(FooClassWithEnum(1, FooEnum.E1), "case class with Int and scala Enum") encodeDecodeTest(FooEnum.E1, "scala Enum") @@ -473,8 +561,9 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes private def encodeDecodeTest[T : ExpressionEncoder]( input: T, - testName: String): Unit = { - testAndVerifyNotLeakingReflectionObjects(s"encode/decode for $testName: $input") { + testName: String, + useFallback: Boolean = false): Unit = { + testAndVerifyNotLeakingReflectionObjects(s"encode/decode for $testName: $input", useFallback) { val encoder = implicitly[ExpressionEncoder[T]] // Make sure encoder is serializable. @@ -505,10 +594,8 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes val isCorrect = (input, convertedBack) match { case (b1: Array[Byte], b2: Array[Byte]) => Arrays.equals(b1, b2) case (b1: Array[Int], b2: Array[Int]) => Arrays.equals(b1, b2) - case (b1: Array[Array[_]], b2: Array[Array[_]]) => - Arrays.deepEquals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]]) case (b1: Array[_], b2: Array[_]) => - Arrays.equals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]]) + Arrays.deepEquals(b1.asInstanceOf[Array[AnyRef]], b2.asInstanceOf[Array[AnyRef]]) case (left: Comparable[_], right: Comparable[_]) => left.asInstanceOf[Comparable[Any]].compareTo(right) == 0 case _ => input == convertedBack @@ -568,9 +655,16 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes r } - private def testAndVerifyNotLeakingReflectionObjects(testName: String)(testFun: => Any): Unit = { - test(testName) { - verifyNotLeakingReflectionObjects(testFun) + private def testAndVerifyNotLeakingReflectionObjects( + testName: String, useFallback: Boolean = false)(testFun: => Any): Unit = { + if (useFallback) { + testFallback(testName) { + verifyNotLeakingReflectionObjects(testFun) + } + } else { + test(testName) { + verifyNotLeakingReflectionObjects(testFun) + } } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index e46599dc19a8b..d435aa0eb4c05 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -961,26 +961,23 @@ abstract class AnsiCastSuiteBase extends CastSuiteBase { } test("ANSI mode: cast string to timestamp with parse error") { - val activeConf = conf - new ParVector(ALL_TIMEZONES.toVector).foreach { zid => + ALL_TIMEZONES.foreach { zid => def checkCastWithParseError(str: String): Unit = { checkExceptionInExpression[DateTimeException]( cast(Literal(str), TimestampType, Option(zid.getId)), s"Cannot cast $str to TimestampType.") } - SQLConf.withExistingConf(activeConf) { - checkCastWithParseError("123") - checkCastWithParseError("2015-03-18 123142") - checkCastWithParseError("2015-03-18T123123") - checkCastWithParseError("2015-03-18X") - checkCastWithParseError("2015/03/18") - checkCastWithParseError("2015.03.18") - checkCastWithParseError("20150318") - checkCastWithParseError("2015-031-8") - checkCastWithParseError("2015-03-18T12:03:17-0:70") - checkCastWithParseError("abdef") - } + checkCastWithParseError("123") + checkCastWithParseError("2015-03-18 123142") + checkCastWithParseError("2015-03-18T123123") + checkCastWithParseError("2015-03-18X") + checkCastWithParseError("2015/03/18") + checkCastWithParseError("2015.03.18") + checkCastWithParseError("20150318") + checkCastWithParseError("2015-031-8") + checkCastWithParseError("2015-03-18T12:03:17-0:70") + checkCastWithParseError("abdef") } } @@ -1510,6 +1507,10 @@ class CastSuite extends CastSuiteBase { test("Cast from double II") { checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) } + + test("SPARK-34727: cast from float II") { + checkCast(16777215.0f, java.time.Instant.ofEpochSecond(16777215)) + } } /** 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 79770505ec35d..763ecba2f3fcb 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 @@ -1365,6 +1365,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( SecondsToTimestamp(Literal(123.456789123)), Instant.ofEpochSecond(123, 456789000)) + checkEvaluation(SecondsToTimestamp(Literal(16777215.0f)), Instant.ofEpochSecond(16777215)) } test("TIMESTAMP_MILLIS") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala index 8f030b45e5d3e..ba3dc42967fb9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.util.IntervalUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.types.UTF8String class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -48,8 +49,10 @@ class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { testBothCodegenAndInterpreted("unsafe buffer") { val inputRow = InternalRow.fromSeq(Seq(false, 1.toByte, 9.toShort, -18, 53L, 3.2f, 7.8, 4, 9L)) - val numBytes = UnsafeRow.calculateBitSetWidthInBytes(fixedLengthTypes.length) - val unsafeBuffer = UnsafeRow.createFromByteArray(numBytes, fixedLengthTypes.length) + val numFields = fixedLengthTypes.length + val numBytes = Platform.BYTE_ARRAY_OFFSET + UnsafeRow.calculateBitSetWidthInBytes(numFields) + + UnsafeRow.WORD_SIZE * numFields + val unsafeBuffer = UnsafeRow.createFromByteArray(numBytes, numFields) val proj = createMutableProjection(fixedLengthTypes) val projUnsafeRow = proj.target(unsafeBuffer)(inputRow) assert(SafeProjection.create(fixedLengthTypes)(projUnsafeRow) === inputRow) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index bc2b93e5390da..35ceff6931d7e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -608,6 +608,53 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkExceptionInExpression[RuntimeException]( serializer4, EmptyRow, "Cannot use null as map key!") } + + test("SPARK-35244: invoke should throw the original exception") { + val strClsType = ObjectType(classOf[String]) + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(3))), "") + + val mathCls = classOf[Math] + checkExceptionInExpression[ArithmeticException]( + StaticInvoke(mathCls, IntegerType, "addExact", Seq(Literal(Int.MaxValue), Literal(1))), "") + } + + test("SPARK-35278: invoke should find method with correct number of parameters") { + val strClsType = ObjectType(classOf[String]) + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(3))), "") + + checkObjectExprEvaluation( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0))), "a") + + checkExceptionInExpression[StringIndexOutOfBoundsException]( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0), Literal(3))), "") + + checkObjectExprEvaluation( + Invoke(Literal("a", strClsType), "substring", strClsType, Seq(Literal(0), Literal(1))), "a") + } + + test("SPARK-35278: invoke should correctly invoke override method") { + val clsType = ObjectType(classOf[ConcreteClass]) + val obj = new ConcreteClass + + val input = (1, 2) + checkObjectExprEvaluation( + Invoke(Literal(obj, clsType), "testFunc", IntegerType, + Seq(Literal(input, ObjectType(input.getClass)))), 2) + } + + test("SPARK-35288: static invoke should find method without exact param type match") { + val input = (1, 2) + + checkObjectExprEvaluation( + StaticInvoke(TestStaticInvoke.getClass, IntegerType, "func", + Seq(Literal(input, ObjectType(input.getClass)))), 3) + + checkObjectExprEvaluation( + StaticInvoke(TestStaticInvoke.getClass, IntegerType, "func", + Seq(Literal(1, IntegerType))), -1) + } } class TestBean extends Serializable { @@ -618,3 +665,23 @@ class TestBean extends Serializable { def setNonPrimitive(i: AnyRef): Unit = assert(i != null, "this setter should not be called with null.") } + +object TestStaticInvoke { + def func(param: Any): Int = param match { + case pair: Tuple2[_, _] => + pair.asInstanceOf[Tuple2[Int, Int]]._1 + pair.asInstanceOf[Tuple2[Int, Int]]._2 + case _ => -1 + } +} + +abstract class BaseClass[T] { + def testFunc(param: T): Int +} + +class ConcreteClass extends BaseClass[Product] with Serializable { + override def testFunc(param: Product): Int = param match { + case _: Tuple2[_, _] => 2 + case _: Tuple3[_, _, _] => 3 + case _ => 4 + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala index 8d7501d952ecb..019857580d077 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RegexpExpressionsSuite.scala @@ -22,6 +22,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.optimizer.ConstantFolding +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StringType /** @@ -470,4 +472,12 @@ class RegexpExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { cache.setAccessible(true) assert(cache.get(expr).asInstanceOf[java.util.regex.Pattern].pattern().contains("a")) } + + test("SPARK-34814: LikeSimplification should handle NULL") { + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> + ConstantFolding.getClass.getName.stripSuffix("$")) { + checkEvaluation(Literal.create("foo", StringType) + .likeAll("%foo%", Literal.create(null, StringType)), null) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala index c04f59ebb1b1b..2fab553183492 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala @@ -18,28 +18,36 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf.CASE_SENSITIVE import org.apache.spark.sql.types._ -class SchemaPruningSuite extends SparkFunSuite { - test("prune schema by the requested fields") { - def testPrunedSchema( - schema: StructType, - requestedFields: StructField*): Unit = { - val requestedRootFields = requestedFields.map { f => - // `derivedFromAtt` doesn't affect the result of pruned schema. - SchemaPruning.RootField(field = f, derivedFromAtt = true) - } - val expectedSchema = SchemaPruning.pruneDataSchema(schema, requestedRootFields) - assert(expectedSchema == StructType(requestedFields)) +class SchemaPruningSuite extends SparkFunSuite with SQLHelper { + private def testPrunedSchema( + schema: StructType, + requestedFields: Seq[StructField], + expectedSchema: StructType): Unit = { + val requestedRootFields = requestedFields.map { f => + // `derivedFromAtt` doesn't affect the result of pruned schema. + SchemaPruning.RootField(field = f, derivedFromAtt = true) } + val prunedSchema = SchemaPruning.pruneDataSchema(schema, requestedRootFields) + assert(prunedSchema === expectedSchema) + } - testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("a", IntegerType)) - testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("b", IntegerType)) + test("prune schema by the requested fields") { + testPrunedSchema( + StructType.fromDDL("a int, b int"), + Seq(StructField("a", IntegerType)), + StructType.fromDDL("a int, b int")) val structOfStruct = StructType.fromDDL("a struct, b int") - testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("a int, b int"))) - testPrunedSchema(structOfStruct, StructField("b", IntegerType)) - testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("b int"))) + testPrunedSchema(structOfStruct, + Seq(StructField("a", StructType.fromDDL("a int")), StructField("b", IntegerType)), + StructType.fromDDL("a struct, b int")) + testPrunedSchema(structOfStruct, + Seq(StructField("a", StructType.fromDDL("a int"))), + StructType.fromDDL("a struct, b int")) val arrayOfStruct = StructField("a", ArrayType(StructType.fromDDL("a int, b int, c string"))) val mapOfStruct = StructField("d", MapType(StructType.fromDDL("a int, b int, c string"), @@ -49,14 +57,76 @@ class SchemaPruningSuite extends SparkFunSuite { arrayOfStruct :: StructField("b", structOfStruct) :: StructField("c", IntegerType) :: mapOfStruct :: Nil) - testPrunedSchema(complexStruct, StructField("a", ArrayType(StructType.fromDDL("b int"))), - StructField("b", StructType.fromDDL("a int"))) testPrunedSchema(complexStruct, - StructField("a", ArrayType(StructType.fromDDL("b int, c string"))), - StructField("b", StructType.fromDDL("b int"))) + Seq(StructField("a", ArrayType(StructType.fromDDL("b int"))), + StructField("b", StructType.fromDDL("a int"))), + StructType( + StructField("a", ArrayType(StructType.fromDDL("b int"))) :: + StructField("b", StructType.fromDDL("a int")) :: + StructField("c", IntegerType) :: + mapOfStruct :: Nil)) + testPrunedSchema(complexStruct, + Seq(StructField("a", ArrayType(StructType.fromDDL("b int, c string"))), + StructField("b", StructType.fromDDL("b int"))), + StructType( + StructField("a", ArrayType(StructType.fromDDL("b int, c string"))) :: + StructField("b", StructType.fromDDL("b int")) :: + StructField("c", IntegerType) :: + mapOfStruct :: Nil)) val selectFieldInMap = StructField("d", MapType(StructType.fromDDL("a int, b int"), StructType.fromDDL("e int, f string"))) - testPrunedSchema(complexStruct, StructField("c", IntegerType), selectFieldInMap) + testPrunedSchema(complexStruct, + Seq(StructField("c", IntegerType), selectFieldInMap), + StructType( + arrayOfStruct :: + StructField("b", structOfStruct) :: + StructField("c", IntegerType) :: + selectFieldInMap :: Nil)) + } + + test("SPARK-35096: test case insensitivity of pruned schema") { + val upperCaseSchema = StructType.fromDDL("A struct, B int") + val lowerCaseSchema = StructType.fromDDL("a struct, b int") + val upperCaseRequestedFields = Seq(StructField("A", StructType.fromDDL("A int"))) + val lowerCaseRequestedFields = Seq(StructField("a", StructType.fromDDL("a int"))) + + Seq(true, false).foreach { isCaseSensitive => + withSQLConf(CASE_SENSITIVE.key -> isCaseSensitive.toString) { + if (isCaseSensitive) { + testPrunedSchema( + upperCaseSchema, + upperCaseRequestedFields, + StructType.fromDDL("A struct, B int")) + testPrunedSchema( + upperCaseSchema, + lowerCaseRequestedFields, + upperCaseSchema) + + testPrunedSchema( + lowerCaseSchema, + upperCaseRequestedFields, + lowerCaseSchema) + testPrunedSchema( + lowerCaseSchema, + lowerCaseRequestedFields, + StructType.fromDDL("a struct, b int")) + } else { + Seq(upperCaseRequestedFields, lowerCaseRequestedFields).foreach { requestedFields => + testPrunedSchema( + upperCaseSchema, + requestedFields, + StructType.fromDDL("A struct, B int")) + } + + Seq(upperCaseRequestedFields, lowerCaseRequestedFields).foreach { requestedFields => + testPrunedSchema( + lowerCaseSchema, + requestedFields, + StructType.fromDDL("a struct, b int")) + } + } + } + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala index 0147c6c6a8260..65671d253dc53 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{DataType, IntegerType} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType} -class SubexpressionEliminationSuite extends SparkFunSuite { +class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHelper { test("Semantic equals and hash") { val a: AttributeReference = AttributeReference("name", IntegerType)() val id = { @@ -253,6 +254,61 @@ class SubexpressionEliminationSuite extends SparkFunSuite { assert(equivalence2.getAllEquivalentExprs.count(_.size == 2) == 0) } + + test("SPARK-34723: Correct parameter type for subexpression elimination under whole-stage") { + withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> "1") { + val str = BoundReference(0, BinaryType, false) + val pos = BoundReference(1, IntegerType, false) + + val substr = new Substring(str, pos) + + val add = Add(Length(substr), Literal(1)) + val add2 = Add(Length(substr), Literal(2)) + + val ctx = new CodegenContext() + val exprs = Seq(add, add2) + + val oneVar = ctx.freshVariable("str", BinaryType) + val twoVar = ctx.freshVariable("pos", IntegerType) + ctx.addMutableState("byte[]", oneVar, forceInline = true, useFreshName = false) + ctx.addMutableState("int", twoVar, useFreshName = false) + + ctx.INPUT_ROW = null + ctx.currentVars = Seq( + ExprCode(TrueLiteral, oneVar), + ExprCode(TrueLiteral, twoVar)) + + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(exprs) + ctx.withSubExprEliminationExprs(subExprs.states) { + exprs.map(_.genCode(ctx)) + } + val subExprsCode = subExprs.codes.mkString("\n") + + val codeBody = s""" + public java.lang.Object generate(Object[] references) { + return new TestCode(references); + } + + class TestCode { + ${ctx.declareMutableStates()} + + public TestCode(Object[] references) { + } + + public void initialize(int partitionIndex) { + ${subExprsCode} + } + + ${ctx.declareAddedFunctions()} + } + """ + + val code = CodeFormatter.stripOverlappingComments( + new CodeAndComment(codeBody, ctx.getPlaceHolderToComments())) + + CodeGenerator.compile(code) + } + } } case class CodegenFallbackExpression(child: Expression) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala index cf875efc62c98..c8c1ecd7718b0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala @@ -74,7 +74,7 @@ class OptimizeLimitZeroSuite extends PlanTest { ).foreach { case (jt, correctAnswer) => test(s"Limit 0: for join type $jt") { val query = testRelation1 - .join(testRelation2.limit(0), joinType = jt, condition = Some('a.attr == 'b.attr)) + .join(testRelation2.limit(0), joinType = jt, condition = Some('a.attr === 'b.attr)) val optimized = Optimize.execute(query.analyze) @@ -86,9 +86,9 @@ class OptimizeLimitZeroSuite extends PlanTest { val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = Seq(Row(1))) val subJoinQuery = testRelation1 - .join(testRelation2, joinType = Inner, condition = Some('a.attr == 'b.attr)) + .join(testRelation2, joinType = Inner, condition = Some('a.attr === 'b.attr)) val query = subJoinQuery - .join(testRelation3.limit(0), joinType = Inner, condition = Some('a.attr == 'c.attr)) + .join(testRelation3.limit(0), joinType = Inner, condition = Some('a.attr === 'c.attr)) val optimized = Optimize.execute(query.analyze) val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala index b093b39cc4b88..e63742ac0de56 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeWithFieldsSuite.scala @@ -126,4 +126,25 @@ class OptimizeWithFieldsSuite extends PlanTest { comparePlans(optimized, correctAnswer) } } + + test("SPARK-35213: ensure optimize WithFields maintains correct WithField ordering") { + val originalQuery = testRelation + .select( + Alias(UpdateFields('a, + WithField("a1", Literal(3)) :: + WithField("b1", Literal(4)) :: + WithField("a1", Literal(5)) :: + Nil), "out")()) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .select( + Alias(UpdateFields('a, + WithField("a1", Literal(5)) :: + WithField("b1", Literal(4)) :: + Nil), "out")()) + .analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index 5c980abdd8f53..54c692c13b781 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -147,7 +147,7 @@ class PropagateEmptyRelationSuite extends PlanTest { testcases.foreach { case (left, right, jt, answer) => val query = testRelation1 .where(left) - .join(testRelation2.where(right), joinType = jt, condition = Some('a.attr == 'b.attr)) + .join(testRelation2.where(right), joinType = jt, condition = Some('a.attr === 'b.attr)) val optimized = Optimize.execute(query.analyze) val correctAnswer = answer.getOrElse(OptimizeWithoutPropagateEmptyRelation.execute(query.analyze)) 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 2e1cf4a137e25..7b591766cb4b2 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 @@ -369,9 +369,6 @@ class JoinReorderSuite extends JoinReorderPlanTestBase with StatsEstimationTestB 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)) 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 a42914765dcc8..c4a660532b207 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,13 @@ class StarJoinCostBasedReorderSuite extends JoinReorderPlanTestBase with StatsEs (nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) val expected = - t3.join(t4, Inner, Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) + 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("d1_c2") === 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/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 9f6a76b9228c5..9711cdc559c5c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -425,7 +425,7 @@ class ExpressionParserSuite extends AnalysisTest { assertEqual("(a + b).b", ('a + 'b).getField("b")) // This will fail analysis. assertEqual( "struct(a, b).b", - namedStruct(NamePlaceholder, 'a, NamePlaceholder, 'b).getField("b")) + namedStruct(Literal("a"), 'a, Literal("b"), 'b).getField("b")) } test("reference") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7c70ab98e4183..3676f0cfd73dc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -44,12 +44,18 @@ trait CodegenInterpretedPlanTest extends PlanTest { val codegenMode = CodegenObjectFactoryMode.CODEGEN_ONLY.toString val interpretedMode = CodegenObjectFactoryMode.NO_CODEGEN.toString - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { - super.test(testName + " (codegen path)", testTags: _*)(testFun)(pos) - } - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) { - super.test(testName + " (interpreted path)", testTags: _*)(testFun)(pos) - } + super.test(testName + " (codegen path)", testTags: _*)( + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { testFun })(pos) + super.test(testName + " (interpreted path)", testTags: _*)( + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> interpretedMode) { testFun })(pos) + } + + protected def testFallback( + testName: String, + testTags: Tag*)(testFun: => Any)(implicit pos: source.Position): Unit = { + val codegenMode = CodegenObjectFactoryMode.FALLBACK.toString + super.test(testName + " (codegen fallback mode)", testTags: _*)( + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { testFun })(pos) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 4ad8475a0113c..d837af70ae104 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -594,6 +594,27 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper { "class" -> classOf[JsonTestTreeNode].getName, "num-children" -> 0, "arg" -> "1"))) + + // Convert Seq of Product contains TreeNode to JSON. + assertJSON( + Seq(("a", JsonTestTreeNode("0")), ("b", JsonTestTreeNode("1"))), + List( + JObject( + "product-class" -> "scala.Tuple2", + "_1" -> "a", + "_2" -> List(JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "0" + ))), + JObject( + "product-class" -> "scala.Tuple2", + "_1" -> "b", + "_2" -> List(JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "1" + ))))) } test("toJSON should not throws java.lang.StackOverflowError") { 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 3d841f32379ff..fb2d5114536b3 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 @@ -471,6 +471,13 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("2011-12-25 09:00:00.123456", JST.getId, "2011-12-25 18:00:00.123456") test("2011-12-25 09:00:00.123456", LA.getId, "2011-12-25 01:00:00.123456") test("2011-12-25 09:00:00.123456", "Asia/Shanghai", "2011-12-25 17:00:00.123456") + test("2011-12-25 09:00:00.123456", "-7", "2011-12-25 02:00:00.123456") + test("2011-12-25 09:00:00.123456", "+8:00", "2011-12-25 17:00:00.123456") + test("2011-12-25 09:00:00.123456", "+8:00:00", "2011-12-25 17:00:00.123456") + test("2011-12-25 09:00:00.123456", "+0800", "2011-12-25 17:00:00.123456") + test("2011-12-25 09:00:00.123456", "-071020", "2011-12-25 01:49:40.123456") + test("2011-12-25 09:00:00.123456", "-07:10:20", "2011-12-25 01:49:40.123456") + } } @@ -496,6 +503,12 @@ class DateTimeUtilsSuite extends SparkFunSuite with Matchers with SQLHelper { test("2011-12-25 18:00:00.123456", JST.getId, "2011-12-25 09:00:00.123456") test("2011-12-25 01:00:00.123456", LA.getId, "2011-12-25 09:00:00.123456") test("2011-12-25 17:00:00.123456", "Asia/Shanghai", "2011-12-25 09:00:00.123456") + test("2011-12-25 02:00:00.123456", "-7", "2011-12-25 09:00:00.123456") + test("2011-12-25 17:00:00.123456", "+8:00", "2011-12-25 09:00:00.123456") + test("2011-12-25 17:00:00.123456", "+8:00:00", "2011-12-25 09:00:00.123456") + test("2011-12-25 17:00:00.123456", "+0800", "2011-12-25 09:00:00.123456") + test("2011-12-25 01:49:40.123456", "-071020", "2011-12-25 09:00:00.123456") + test("2011-12-25 01:49:40.123456", "-07:10:20", "2011-12-25 09:00:00.123456") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala index ec73f4518737d..eb257b7975622 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala @@ -40,6 +40,10 @@ class NumberConverterSuite extends SparkFunSuite { checkConv("11abc", 10, 16, "B") } + test("SPARK-34909: convert negative to unsigned") { + checkConv("-10", 11, 7, "45012021522523134134555") + } + test("byte to binary") { checkToBinary(0.toByte) checkToBinary(1.toByte) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.scala new file mode 100644 index 0000000000000..b8c60dfbf4f97 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/PhysicalAggregationSuite.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.catalyst.util + +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalAggregation +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation + +class PhysicalAggregationSuite extends PlanTest { + val testRelation = LocalRelation('a.int, 'b.int) + + test("SPARK-35014: a foldable expression should not be replaced by an AttributeReference") { + val query = testRelation + .groupBy('a, Literal.create(1) as 'k)( + 'a, Round(Literal.create(1.2), Literal.create(1)) as 'r, count('b) as 'c) + val analyzedQuery = SimpleAnalyzer.execute(query) + + val PhysicalAggregation( + groupingExpressions, + aggregateExpressions, + resultExpressions, + _ /* child */ + ) = analyzedQuery + + assertResult(2)(groupingExpressions.length) + assertResult(1)(aggregateExpressions.length) + assertResult(3)(resultExpressions.length) + + // Verify that Round's scale parameter is a Literal. + resultExpressions(1) match { + case Alias(Round(_, _: Literal), _) => + case other => fail("unexpected result expression: " + other) + } + } +} diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 17eff4a549926..a4043d9cb3a7b 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index bbf0ac1dd85e9..308bb96502b19 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -395,10 +395,13 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { private def fillMap(values: Seq[(String, Any)]): DataFrame = { // Error handling - values.foreach { case (colName, replaceValue) => + val attrToValue = AttributeMap(values.map { case (colName, replaceValue) => // Check column name exists - df.resolve(colName) - + val attr = df.resolve(colName) match { + case a: Attribute => a + case _ => throw new UnsupportedOperationException( + s"Nested field ${colName} is not supported.") + } // Check data type replaceValue match { case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long | _: jl.Boolean | _: String => @@ -406,31 +409,29 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { case _ => throw new IllegalArgumentException( s"Unsupported value type ${replaceValue.getClass.getName} ($replaceValue).") } - } - - val columnEquals = df.sparkSession.sessionState.analyzer.resolver - val projections = df.schema.fields.map { f => - values.find { case (k, _) => columnEquals(k, f.name) }.map { case (_, v) => - v match { - case v: jl.Float => fillCol[Float](f, v) - case v: jl.Double => fillCol[Double](f, v) - case v: jl.Long => fillCol[Long](f, v) - case v: jl.Integer => fillCol[Integer](f, v) - case v: jl.Boolean => fillCol[Boolean](f, v.booleanValue()) - case v: String => fillCol[String](f, v) - } - }.getOrElse(df.col(f.name)) + attr -> replaceValue + }) + + val output = df.queryExecution.analyzed.output + val projections = output.map { + attr => attrToValue.get(attr).map { + case v: jl.Float => fillCol[Float](attr, v) + case v: jl.Double => fillCol[Double](attr, v) + case v: jl.Long => fillCol[Long](attr, v) + case v: jl.Integer => fillCol[Integer](attr, v) + case v: jl.Boolean => fillCol[Boolean](attr, v.booleanValue()) + case v: String => fillCol[String](attr, v) + }.getOrElse(Column(attr)) } df.select(projections : _*) } /** - * Returns a [[Column]] expression that replaces null value in `col` with `replacement`. - * It selects a column based on its name. + * Returns a [[Column]] expression that replaces null value in column defined by `attr` + * with `replacement`. */ - private def fillCol[T](col: StructField, replacement: T): Column = { - val quotedColName = "`" + col.name + "`" - fillCol(col.dataType, col.name, df.col(quotedColName), replacement) + private def fillCol[T](attr: Attribute, replacement: T): Column = { + fillCol(attr.dataType, attr.name, Column(attr), replacement) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index b94c42a2c9544..50f8aa0aa796b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -73,8 +73,10 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 1.4.0 */ def schema(schema: StructType): DataFrameReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -90,10 +92,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * @since 2.3.0 */ def schema(schemaString: String): DataFrameReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** @@ -744,7 +743,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
  • `STOP_AT_DELIMITER`: If unescaped quotes are found in the input, consider the value * as an unquoted value. This will make the parser accumulate all characters until the * delimiter or a line ending is found in the input.
  • - *
  • `STOP_AT_DELIMITER`: If unescaped quotes are found in the input, the content parsed + *
  • `SKIP_VALUE`: If unescaped quotes are found in the input, the content parsed * for the given value will be skipped and the value set in nullValue will be produced * instead.
  • *
  • `RAISE_ERROR`: If unescaped quotes are found in the input, a TextParsingException diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index c40ce0f4777c6..7e735eecbac3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} +import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedFunction} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -80,11 +80,7 @@ class RelationalGroupedDataset protected[sql]( } } - // Wrap UnresolvedAttribute with UnresolvedAlias, as when we resolve UnresolvedAttribute, we - // will remove intermediate Alias for ExtractValue chain, and we need to alias it again to - // make it a NamedExpression. private[this] def alias(expr: Expression): NamedExpression = expr match { - case u: UnresolvedAttribute => UnresolvedAlias(u) case expr: NamedExpression => expr case a: AggregateExpression if a.aggregateFunction.isInstanceOf[TypedAggregateExpression] => UnresolvedAlias(a, Some(Column.generateAlias)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 50e9deea44755..dae1bc474ad28 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import java.io.Closeable +import java.util.UUID import java.util.concurrent.TimeUnit._ import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} @@ -102,6 +103,8 @@ class SparkSession private( new SparkSessionExtensions), Map.empty) } + private[sql] val sessionUUID: String = UUID.randomUUID.toString + sparkContext.assertNotStopped() // If there is no active SparkSession, uses the default SQL conf. Otherwise, use the session's. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 53da69e709e90..0e8241909efb4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -216,12 +216,19 @@ class ResolveSessionCatalog( case AlterViewUnsetPropertiesStatement(SessionCatalogAndTable(_, tbl), keys, ifExists) => AlterTableUnsetPropertiesCommand(tbl.asTableIdentifier, keys, ifExists, isView = true) - case d @ DescribeNamespace(SessionCatalogAndNamespace(_, ns), _) => + case DescribeNamespace(SessionCatalogAndNamespace(_, ns), extended, output) => if (ns.length != 1) { throw new AnalysisException( s"The database name is not valid: ${ns.quoted}") } - DescribeDatabaseCommand(ns.head, d.extended) + val newOutput = if (conf.getConf(SQLConf.LEGACY_KEEP_COMMAND_OUTPUT_SCHEMA)) { + assert(output.length == 2) + Seq(output.head.withName("database_description_item"), + output.last.withName("database_description_value")) + } else { + output + } + DescribeDatabaseCommand(ns.head, extended, newOutput) case AlterNamespaceSetProperties(SessionCatalogAndNamespace(_, ns), properties) => if (ns.length != 1) { @@ -250,7 +257,7 @@ class ResolveSessionCatalog( AlterTableRenameCommand(oldName.asTableIdentifier, newName.asTableIdentifier, isView) // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet. - case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended) => + case DescribeRelation(ResolvedV1TableOrViewIdentifier(ident), partitionSpec, isExtended, _) => DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended) case DescribeColumn(ResolvedV1TableOrViewIdentifier(ident), colNameParts, isExtended) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala index 89d3b53510469..596c8b3282dc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -418,7 +418,8 @@ case class AdaptiveSparkPlanExec( // Check the `stageCache` again for reuse. If a match is found, ditch the new stage // and reuse the existing stage found in the `stageCache`, otherwise update the // `stageCache` with the new stage. - val queryStage = context.stageCache.getOrElseUpdate(e.canonicalized, newStage) + val queryStage = context.stageCache.getOrElseUpdate( + newStage.plan.canonicalized, newStage) if (queryStage.ne(newStage)) { newStage = reuseQueryStage(queryStage, e) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala index 49a4c25fa637f..2319c9ed1cfd3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/CustomShuffleReaderExec.scala @@ -179,12 +179,12 @@ case class CustomShuffleReaderExec private( } private lazy val shuffleRDD: RDD[_] = { - sendDriverMetrics() - - shuffleStage.map { stage => - stage.shuffle.getShuffleRDD(partitionSpecs.toArray) - }.getOrElse { - throw new IllegalStateException("operating on canonicalized plan") + shuffleStage match { + case Some(stage) => + sendDriverMetrics() + stage.shuffle.getShuffleRDD(partitionSpecs.toArray) + case _ => + throw new IllegalStateException("operating on canonicalized plan") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala index 085934d906b3c..82dc9a039f65c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala @@ -63,8 +63,8 @@ object OptimizeSkewedJoin extends CustomShuffleReaderRule { /** * A partition is considered as a skewed partition if its size is larger than the median - * partition size * ADAPTIVE_EXECUTION_SKEWED_PARTITION_FACTOR and also larger than - * ADVISORY_PARTITION_SIZE_IN_BYTES. + * partition size * SKEW_JOIN_SKEWED_PARTITION_FACTOR and also larger than + * SKEW_JOIN_SKEWED_PARTITION_THRESHOLD. */ private def isSkewed(size: Long, medianSize: Long): Boolean = { size > medianSize * conf.getConf(SQLConf.SKEW_JOIN_SKEWED_PARTITION_FACTOR) && diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala index a56007f5d5d95..c9de8c7e1a9d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala @@ -19,12 +19,13 @@ package org.apache.spark.sql.execution.command import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkContext import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.util.SerializableConfiguration /** @@ -73,4 +74,26 @@ object DataWritingCommand { attr.withName(outputName) } } + + /** + * When execute CTAS operators, Spark will use [[InsertIntoHadoopFsRelationCommand]] + * or [[InsertIntoHiveTable]] command to write data, they both inherit metrics from + * [[DataWritingCommand]], but after running [[InsertIntoHadoopFsRelationCommand]] + * or [[InsertIntoHiveTable]], we only update metrics in these two command through + * [[BasicWriteJobStatsTracker]], we also need to propogate metrics to the command + * that actually calls [[InsertIntoHadoopFsRelationCommand]] or [[InsertIntoHiveTable]]. + * + * @param sparkContext Current SparkContext. + * @param command Command to execute writing data. + * @param metrics Metrics of real DataWritingCommand. + */ + def propogateMetrics( + sparkContext: SparkContext, + command: DataWritingCommand, + metrics: Map[String, SQLMetric]): Unit = { + command.metrics.foreach { case (key, metric) => metrics(key).set(metric.value) } + SQLMetrics.postDriverMetricUpdates(sparkContext, + sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY), + metrics.values.toSeq) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index 7d92e6e189fb2..888fb33764588 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -138,7 +138,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) extends RunnableComm s"showing ${SQLConf.SHUFFLE_PARTITIONS.key} instead.") Seq(Row( SQLConf.SHUFFLE_PARTITIONS.key, - sparkSession.sessionState.conf.numShufflePartitions.toString)) + sparkSession.sessionState.conf.defaultNumShufflePartitions.toString)) } (keyValueOutput, runFunc) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index be7fa7b1b447e..dc26e00599aea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -217,7 +217,7 @@ case class CreateDataSourceTableAsSelectCommand( catalogTable = if (tableExists) Some(table) else None) try { - dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan) + dataSource.writeAndRead(mode, query, outputColumnNames, physicalPlan, metrics) } catch { case ex: AnalysisException => logError(s"Failed to write to table ${table.identifier.unquotedString}", ex) 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 2f96c453a6e30..10af8f34b6b26 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 @@ -35,7 +35,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} import org.apache.spark.sql.connector.catalog.SupportsNamespaces._ @@ -169,7 +169,8 @@ case class AlterDatabaseSetLocationCommand(databaseName: String, location: Strin */ case class DescribeDatabaseCommand( databaseName: String, - extended: Boolean) + extended: Boolean, + override val output: Seq[Attribute]) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { @@ -195,11 +196,6 @@ case class DescribeDatabaseCommand( result } } - - override val output: Seq[Attribute] = { - AttributeReference("database_description_item", StringType, nullable = false)() :: - AttributeReference("database_description_value", StringType, nullable = false)() :: Nil - } } /** @@ -828,7 +824,12 @@ case class AlterTableSetLocationCommand( DDLUtils.verifyPartitionProviderIsHive( sparkSession, table, "ALTER TABLE ... SET LOCATION") // Partition spec is specified, so we set the location only for this partition - val part = catalog.getPartition(table.identifier, spec) + val normalizedSpec = PartitioningUtils.normalizePartitionSpec( + spec, + table.partitionSchema, + table.identifier.quotedString, + sparkSession.sessionState.conf.resolver) + val part = catalog.getPartition(table.identifier, normalizedSpec) val newPart = part.copy(storage = part.storage.copy(locationUri = Some(locUri))) catalog.alterPartitions(table.identifier, Seq(newPart)) case None => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala index 330a503e5f8e2..07c814889fe56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala @@ -116,7 +116,8 @@ case class DescribeFunctionCommand( } override def run(sparkSession: SparkSession): Seq[Row] = { - // Hard code "<>", "!=", "between", and "case" for now as there is no corresponding functions. + // Hard code "<>", "!=", "between", "case", and "||" + // for now as there is no corresponding functions. functionName.funcName.toLowerCase(Locale.ROOT) match { case "<>" => Row(s"Function: $functionName") :: @@ -136,6 +137,9 @@ case class DescribeFunctionCommand( "[WHEN expr4 THEN expr5]* [ELSE expr6] END - " + "When `expr1` = `expr2`, returns `expr3`; " + "when `expr1` = `expr4`, return `expr5`; else return `expr6`.") :: Nil + case "||" => + Row("Function: ||") :: + Row("Usage: expr1 || expr2 - Returns the concatenation of `expr1` and `expr2`.") :: Nil case _ => try { val info = sparkSession.sessionState.catalog.lookupFunctionInfo(functionName) @@ -225,8 +229,10 @@ case class ShowFunctionsCommand( case (f, "USER") if showUserFunctions => f.unquotedString case (f, "SYSTEM") if showSystemFunctions => f.unquotedString } - // Hard code "<>", "!=", "between", and "case" for now as there is no corresponding functions. - // "<>", "!=", "between", and "case" is SystemFunctions, only show when showSystemFunctions=true + // Hard code "<>", "!=", "between", "case", and "||" + // for now as there is no corresponding functions. + // "<>", "!=", "between", "case", and "||" is SystemFunctions, + // only show when showSystemFunctions=true if (showSystemFunctions) { (functionNames ++ StringUtils.filterPattern(FunctionsCommand.virtualOperators, pattern.getOrElse("*"))) @@ -280,5 +286,5 @@ case class RefreshFunctionCommand( object FunctionsCommand { // operators that do not have corresponding functions. // They should be handled `DescribeFunctionCommand`, `ShowFunctionsCommand` - val virtualOperators = Seq("!=", "<>", "between", "case") + val virtualOperators = Seq("!=", "<>", "between", "case", "||") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index 960fe4ad22836..620553c17d342 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -351,10 +351,21 @@ object ViewHelper { "spark.sql.codegen.", "spark.sql.execution.", "spark.sql.shuffle.", - "spark.sql.adaptive.") + "spark.sql.adaptive.", + SQLConf.ADDITIONAL_REMOTE_REPOSITORIES.key) + private val configAllowList = Seq( + SQLConf.DISABLE_HINTS.key + ) + + /** + * Capture view config either of: + * 1. exists in allowList + * 2. do not exists in denyList + */ private def shouldCaptureConfig(key: String): Boolean = { - !configPrefixDenyList.exists(prefix => key.startsWith(prefix)) + configAllowList.exists(prefix => key.equals(prefix)) || + !configPrefixDenyList.exists(prefix => key.startsWith(prefix)) } import CatalogTable._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index e84f5943d30f0..476174a2cad9f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf @@ -518,7 +519,8 @@ case class DataSource( mode: SaveMode, data: LogicalPlan, outputColumnNames: Seq[String], - physicalPlan: SparkPlan): BaseRelation = { + physicalPlan: SparkPlan, + metrics: Map[String, SQLMetric]): BaseRelation = { val outputColumns = DataWritingCommand.logicalPlanOutputWithNames(data, outputColumnNames) if (outputColumns.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { throw new AnalysisException("Cannot save interval data type into external storage.") @@ -546,6 +548,7 @@ case class DataSource( partitionColumns = resolvedPartCols, outputColumnNames = outputColumnNames) resolved.run(sparkSession, physicalPlan) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, resolved, metrics) // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring copy(userSpecifiedSchema = Some(outputColumns.toStructType.asNullable)).resolveRelation() case _ => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index a0b191e60f376..4ed8943ef46f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -40,6 +40,8 @@ import org.apache.spark.sql.types.{StructField, StructType} case class HadoopFsRelation( location: FileIndex, partitionSchema: StructType, + // The top-level columns in `dataSchema` should match the actual physical file schema, otherwise + // the ORC data source may not work with the by-ordinal mode. dataSchema: StructType, bucketSpec: Option[BucketSpec], fileFormat: FileFormat, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 69123ee7af5b9..947b1e904f96c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCoercion import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec +import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.getPartitionValueString import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.types._ @@ -349,7 +350,7 @@ object PartitioningUtils { */ def getPathFragment(spec: TablePartitionSpec, partitionSchema: StructType): String = { partitionSchema.map { field => - escapePathName(field.name) + "=" + escapePathName(spec(field.name)) + escapePathName(field.name) + "=" + getPartitionValueString(spec(field.name)) }.mkString("/") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 6e8b7ea678264..4cb221a83659c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -208,6 +208,8 @@ class JDBCOptions( val principal = parameters.getOrElse(JDBC_PRINCIPAL, null) val tableComment = parameters.getOrElse(JDBC_TABLE_COMMENT, "").toString + + val refreshKrb5Config = parameters.getOrElse(JDBC_REFRESH_KRB5_CONFIG, "false").toBoolean } class JdbcOptionsInWrite( @@ -263,4 +265,5 @@ object JDBCOptions { val JDBC_KEYTAB = newOption("keytab") val JDBC_PRINCIPAL = newOption("principal") val JDBC_TABLE_COMMENT = newOption("tableComment") + val JDBC_REFRESH_KRB5_CONFIG = newOption("refreshKrb5Config") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index e81add4df960a..fbc69704f1479 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection import java.sql.{Connection, Driver} import java.util.ServiceLoader +import javax.security.auth.login.Configuration import scala.collection.mutable @@ -60,7 +61,15 @@ private[jdbc] object ConnectionProvider extends Logging { "JDBC connection initiated but not exactly one connection provider found which can handle " + s"it. Found active providers: ${filteredProviders.mkString(", ")}") SecurityConfigurationLock.synchronized { - filteredProviders.head.getConnection(driver, options) + // Inside getConnection it's safe to get parent again because SecurityConfigurationLock + // makes sure it's untouched + val parent = Configuration.getConfiguration + try { + filteredProviders.head.getConnection(driver, options) + } finally { + logDebug("Restoring original security configuration") + Configuration.setConfiguration(parent) + } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala index 775c3ae4a533a..060653c5a8b79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProvider.scala @@ -34,7 +34,7 @@ private[sql] class DB2ConnectionProvider extends SecureConnectionProvider { override def getConnection(driver: Driver, options: Map[String, String]): Connection = { val jdbcOptions = new JDBCOptions(options) - setAuthenticationConfigIfNeeded(driver, jdbcOptions) + setAuthenticationConfig(driver, jdbcOptions) UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) .doAs( new PrivilegedExceptionAction[Connection]() { @@ -52,11 +52,4 @@ private[sql] class DB2ConnectionProvider extends SecureConnectionProvider { result.put("KerberosServerPrincipal", options.principal) result } - - override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { - val (parent, configEntry) = getConfigWithAppEntry(driver, options) - if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent, driver, options) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala index e3d3e1a43d510..aa8c9227377c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala @@ -61,7 +61,7 @@ private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider { override def getConnection(driver: Driver, options: Map[String, String]): Connection = { val jdbcOptions = new JDBCOptions(options) - setAuthenticationConfigIfNeeded(driver, jdbcOptions) + setAuthenticationConfig(driver, jdbcOptions) UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) .doAs( new PrivilegedExceptionAction[Connection]() { @@ -79,18 +79,4 @@ private[sql] class MSSQLConnectionProvider extends SecureConnectionProvider { result.put("authenticationScheme", "JavaKerberos") result } - - override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { - val (parent, configEntry) = getConfigWithAppEntry(driver, options) - /** - * Couple of things to mention here (v8.2.2 client): - * 1. MS SQL supports JAAS application name configuration - * 2. MS SQL sets a default JAAS config if "java.security.auth.login.config" is not set - */ - val entryUsesKeytab = configEntry != null && - configEntry.exists(_.getOptions().get("useKeyTab") == "true") - if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent, driver, options) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 29a08d0b5f269..6a53c663a2773 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -26,20 +26,5 @@ private[jdbc] class MariaDBConnectionProvider extends SecureConnectionProvider { override val name: String = "mariadb" - override def appEntry(driver: Driver, options: JDBCOptions): String = - "Krb5ConnectorContext" - - override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { - val (parent, configEntry) = getConfigWithAppEntry(driver, options) - /** - * Couple of things to mention here (v2.5.4 client): - * 1. MariaDB doesn't support JAAS application name configuration - * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" is not set - */ - val entryUsesKeytab = configEntry != null && - configEntry.exists(_.getOptions().get("useKeyTab") == "true") - if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { - setAuthenticationConfig(parent, driver, options) - } - } + override def appEntry(driver: Driver, options: JDBCOptions): String = "Krb5ConnectorContext" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala index 0d43851bb255e..ef8d91b5aa8f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProvider.scala @@ -34,7 +34,7 @@ private[sql] class OracleConnectionProvider extends SecureConnectionProvider { override def getConnection(driver: Driver, options: Map[String, String]): Connection = { val jdbcOptions = new JDBCOptions(options) - setAuthenticationConfigIfNeeded(driver, jdbcOptions) + setAuthenticationConfig(driver, jdbcOptions) UserGroupInformation.loginUserFromKeytabAndReturnUGI(jdbcOptions.principal, jdbcOptions.keytab) .doAs( new PrivilegedExceptionAction[Connection]() { @@ -53,11 +53,4 @@ private[sql] class OracleConnectionProvider extends SecureConnectionProvider { result.put("oracle.net.authentication_services", "(KERBEROS5)"); result } - - override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { - val (parent, configEntry) = getConfigWithAppEntry(driver, options) - if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent, driver, options) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala index f26a11e34dc38..ec9ac66147e99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProvider.scala @@ -32,11 +32,4 @@ private[jdbc] class PostgresConnectionProvider extends SecureConnectionProvider val properties = parseURL.invoke(driver, options.url, null).asInstanceOf[Properties] properties.getProperty("jaasApplicationName", "pgjdbc") } - - override def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit = { - val (parent, configEntry) = getConfigWithAppEntry(driver, options) - if (configEntry == null || configEntry.isEmpty) { - setAuthenticationConfig(parent, driver, options) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala index 80c795957dac8..71c20e11f9274 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/SecureConnectionProvider.scala @@ -40,7 +40,7 @@ private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionPro override def getConnection(driver: Driver, options: Map[String, String]): Connection = { val jdbcOptions = new JDBCOptions(options) - setAuthenticationConfigIfNeeded(driver, jdbcOptions) + setAuthenticationConfig(driver, jdbcOptions) super.getConnection(driver: Driver, options: Map[String, String]) } @@ -49,26 +49,11 @@ private[jdbc] abstract class SecureConnectionProvider extends BasicConnectionPro */ def appEntry(driver: Driver, options: JDBCOptions): String - /** - * Sets database specific authentication configuration when needed. If configuration already set - * then later calls must be no op. When the global JVM security configuration changed then the - * related code parts must be synchronized properly. - */ - def setAuthenticationConfigIfNeeded(driver: Driver, options: JDBCOptions): Unit - - protected def getConfigWithAppEntry( - driver: Driver, - options: JDBCOptions): (Configuration, Array[AppConfigurationEntry]) = { + private[connection] def setAuthenticationConfig(driver: Driver, options: JDBCOptions) = { val parent = Configuration.getConfiguration - (parent, parent.getAppConfigurationEntry(appEntry(driver, options))) - } - - protected def setAuthenticationConfig( - parent: Configuration, - driver: Driver, - options: JDBCOptions) = { val config = new SecureConnectionProvider.JDBCConfiguration( - parent, appEntry(driver, options), options.keytab, options.principal) + parent, appEntry(driver, options), options.keytab, + options.principal, options.refreshKrb5Config) logDebug("Adding database specific security configuration") Configuration.setConfiguration(config) } @@ -79,7 +64,8 @@ object SecureConnectionProvider { parent: Configuration, appEntry: String, keytab: String, - principal: String) extends Configuration { + principal: String, + refreshKrb5Config: Boolean) extends Configuration { val entry = new AppConfigurationEntry( SecurityUtils.getKrb5LoginModuleName(), @@ -89,7 +75,8 @@ object SecureConnectionProvider { "useKeyTab" -> "true", "keyTab" -> keytab, "principal" -> principal, - "debug" -> "true" + "debug" -> "true", + "refreshKrb5Config" -> refreshKrb5Config.toString ).asJava ) 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 643607a905d7a..73f7f7c750309 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 @@ -260,14 +260,14 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil - case desc @ DescribeNamespace(ResolvedNamespace(catalog, ns), extended) => - DescribeNamespaceExec(desc.output, catalog.asNamespaceCatalog, ns, extended) :: Nil + case DescribeNamespace(ResolvedNamespace(catalog, ns), extended, output) => + DescribeNamespaceExec(output, catalog.asNamespaceCatalog, ns, extended) :: Nil - case desc @ DescribeRelation(r: ResolvedTable, partitionSpec, isExtended) => + case DescribeRelation(r: ResolvedTable, partitionSpec, isExtended, output) => if (partitionSpec.nonEmpty) { throw new AnalysisException("DESCRIBE does not support partition for v2 tables.") } - DescribeTableExec(desc.output, r.table, isExtended) :: Nil + DescribeTableExec(output, r.table, isExtended) :: Nil case DescribeColumn(_: ResolvedTable, _, _) => throw new AnalysisException("Describing columns is not supported for v2 tables.") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index 363dd154b5fbb..ac63725b774d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -24,8 +24,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.internal.Logging import org.apache.spark.internal.config.IO_WARNING_LARGEFILETHRESHOLD import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionSet} +import org.apache.spark.sql.catalyst.expressions.{AttributeSet, Expression, ExpressionSet} import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.connector.read.{Batch, InputPartition, Scan, Statistics, SupportsReportStatistics} import org.apache.spark.sql.execution.PartitionedFileUtil import org.apache.spark.sql.execution.datasources._ @@ -84,11 +85,24 @@ trait FileScan extends Scan protected def seqToString(seq: Seq[Any]): String = seq.mkString("[", ", ", "]") + private lazy val (normalizedPartitionFilters, normalizedDataFilters) = { + val output = readSchema().toAttributes + val partitionFilterAttributes = AttributeSet(partitionFilters).map(a => a.name -> a).toMap + val dataFiltersAttributes = AttributeSet(dataFilters).map(a => a.name -> a).toMap + val normalizedPartitionFilters = ExpressionSet(partitionFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => partitionFilterAttributes.getOrElse(a.name, a))))) + val normalizedDataFilters = ExpressionSet(dataFilters.map( + QueryPlan.normalizeExpressions(_, + output.map(a => dataFiltersAttributes.getOrElse(a.name, a))))) + (normalizedPartitionFilters, normalizedDataFilters) + } + override def equals(obj: Any): Boolean = obj match { case f: FileScan => - fileIndex == f.fileIndex && readSchema == f.readSchema - ExpressionSet(partitionFilters) == ExpressionSet(f.partitionFilters) && - ExpressionSet(dataFilters) == ExpressionSet(f.dataFilters) + fileIndex == f.fileIndex && readSchema == f.readSchema && + normalizedPartitionFilters == f.normalizedPartitionFilters && + normalizedDataFilters == f.normalizedDataFilters case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 167ba45b888a3..1f57f17911457 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -81,6 +81,10 @@ object PushDownUtils extends PredicateHelper { relation: DataSourceV2Relation, projects: Seq[NamedExpression], filters: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + val exprs = projects ++ filters + val requiredColumns = AttributeSet(exprs.flatMap(_.references)) + val neededOutput = relation.output.filter(requiredColumns.contains) + scanBuilder match { case r: SupportsPushDownRequiredColumns if SQLConf.get.nestedSchemaPruningEnabled => val rootFields = SchemaPruning.identifyRootFields(projects, filters) @@ -89,14 +93,12 @@ object PushDownUtils extends PredicateHelper { } else { new StructType() } - r.pruneColumns(prunedSchema) + val neededFieldNames = neededOutput.map(_.name).toSet + r.pruneColumns(StructType(prunedSchema.filter(f => neededFieldNames.contains(f.name)))) val scan = r.build() scan -> toOutputAttrs(scan.readSchema(), relation) case r: SupportsPushDownRequiredColumns => - val exprs = projects ++ filters - val requiredColumns = AttributeSet(exprs.flatMap(_.references)) - val neededOutput = relation.output.filter(requiredColumns.contains) r.pruneColumns(neededOutput.toStructType) val scan = r.build() // always project, in case the relation's output has been updated and doesn't match diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala index e30f9b65a2c2c..7fac91a337adc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/PartitionPruning.scala @@ -163,6 +163,7 @@ object PartitionPruning extends Rule[LogicalPlan] with PredicateHelper { case _: BinaryComparison => true case _: In | _: InSet => true case _: StringPredicate => true + case _: MultiLikeBase => true case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index 2a9e15851e9f1..cec1286c98a7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -46,7 +46,7 @@ case class BroadcastHashJoinExec( left: SparkPlan, right: SparkPlan, isNullAwareAntiJoin: Boolean = false) - extends HashJoin with CodegenSupport { + extends HashJoin { if (isNullAwareAntiJoin) { require(leftKeys.length == 1, "leftKeys length should be 1") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 53bd591d98a2e..42219ee615c5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -138,7 +138,13 @@ trait HashJoin extends BaseJoinExec with CodegenSupport { UnsafeProjection.create(streamedBoundKeys) @transient protected[this] lazy val boundCondition = if (condition.isDefined) { - Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + if (joinType == FullOuter && buildSide == BuildLeft) { + // Put join left side before right side. This is to be consistent with + // `ShuffledHashJoinExec.fullOuterJoin`. + Predicate.create(condition.get, buildPlan.output ++ streamedPlan.output).eval _ + } else { + Predicate.create(condition.get, streamedPlan.output ++ buildPlan.output).eval _ + } } else { (r: InternalRow) => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index ddbd0a343ffcf..d03aeadc1af30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -122,14 +122,18 @@ trait BaseLimitExec extends LimitExec with CodegenSupport { } protected override def doProduce(ctx: CodegenContext): String = { - child.asInstanceOf[CodegenSupport].produce(ctx, this) - } - - override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { // The counter name is already obtained by the upstream operators via `limitNotReachedChecks`. // Here we have to inline it to not change its name. This is fine as we won't have many limit // operators in one query. + // + // Note: create counter variable here instead of `doConsume()` to avoid compilation error, + // because upstream operators might not call `doConsume()` here + // (e.g. `HashJoin.codegenInner()`). ctx.addMutableState(CodeGenerator.JAVA_INT, countTerm, forceInline = true, useFreshName = false) + child.asInstanceOf[CodegenSupport].produce(ctx, this) + } + + override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { s""" | if ($countTerm < $limit) { | $countTerm += 1; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index b6d8e59877f17..2ab7262763835 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -46,7 +46,18 @@ case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute] val needConversion = dataTypes.exists(EvaluatePython.needConversionInPython) // enable memo iff we serialize the row with schema (schema and class should be memorized) - val pickle = new Pickler(needConversion) + // pyrolite 4.21+ can lookup objects in its cache by value, but `GenericRowWithSchema` objects, + // that we pass from JVM to Python, don't define their `equals()` to take the type of the + // values or the schema of the row into account. This causes like + // `GenericRowWithSchema(Array(1.0, 1.0), + // StructType(Seq(StructField("_1", DoubleType), StructField("_2", DoubleType))))` + // and + // `GenericRowWithSchema(Array(1, 1), + // StructType(Seq(StructField("_1", IntegerType), StructField("_2", IntegerType))))` + // to be `equal()` and so we need to disable this feature explicitly (`valueCompare=false`). + // Please note that cache by reference is still enabled depending on `needConversion`. + val pickle = new Pickler(/* useMemo = */ needConversion, + /* valueCompare = */ false) // Input iterator to Python: input rows are grouped so we send them in batches to Python. // For each row, add it to the queue. val inputIterator = iter.map { row => 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 6b0d33b819a20..1b145f23fc8a9 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 @@ -314,26 +314,28 @@ abstract class StreamExecution( startLatch.countDown() // While active, repeatedly attempt to run batches. - SparkSession.setActiveSession(sparkSession) - - updateStatusMessage("Initializing sources") - // force initialization of the logical plan so that the sources can be created - logicalPlan - - // Adaptive execution can change num shuffle partitions, disallow - sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") - // Disable cost-based join optimization as we do not want stateful operations to be rearranged - sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false") - offsetSeqMetadata = OffsetSeqMetadata( - batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf) - - if (state.compareAndSet(INITIALIZING, ACTIVE)) { - // Unblock `awaitInitialization` - initializationLatch.countDown() - runActivatedStream(sparkSessionForStream) - updateStatusMessage("Stopped") - } else { - // `stop()` is already called. Let `finally` finish the cleanup. + sparkSessionForStream.withActive { + // Adaptive execution can change num shuffle partitions, disallow + sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") + // Disable cost-based join optimization as we do not want stateful operations + // to be rearranged + sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false") + + updateStatusMessage("Initializing sources") + // force initialization of the logical plan so that the sources can be created + logicalPlan + + offsetSeqMetadata = OffsetSeqMetadata( + batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf) + + if (state.compareAndSet(INITIALIZING, ACTIVE)) { + // Unblock `awaitInitialization` + initializationLatch.countDown() + runActivatedStream(sparkSessionForStream) + updateStatusMessage("Stopped") + } else { + // `stop()` is already called. Let `finally` finish the cleanup. + } } } catch { case e if isInterruptedByStop(e, sparkSession.sparkContext) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index e6b41cdb3eb18..6bc49b6ec0cf6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3644,22 +3644,22 @@ object functions { } private def createLambda(f: Column => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) val function = f(Column(x)).expr LambdaFunction(function, Seq(x)) } private def createLambda(f: (Column, Column) => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) - val y = UnresolvedNamedLambdaVariable(Seq("y")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) + val y = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("y"))) val function = f(Column(x), Column(y)).expr LambdaFunction(function, Seq(x, y)) } private def createLambda(f: (Column, Column, Column) => Column) = { - val x = UnresolvedNamedLambdaVariable(Seq("x")) - val y = UnresolvedNamedLambdaVariable(Seq("y")) - val z = UnresolvedNamedLambdaVariable(Seq("z")) + val x = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("x"))) + val y = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("y"))) + val z = UnresolvedNamedLambdaVariable(Seq(UnresolvedNamedLambdaVariable.freshVarName("z"))) val function = f(Column(x), Column(y), Column(z)).expr LambdaFunction(function, Seq(x, y, z)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index d82fa9e88592f..9b88274a95713 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -64,8 +64,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.0.0 */ def schema(schema: StructType): DataStreamReader = { - val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(replaced) + if (schema != null) { + val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType] + this.userSpecifiedSchema = Option(replaced) + } this } @@ -77,10 +79,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * @since 2.3.0 */ def schema(schemaString: String): DataStreamReader = { - val rawSchema = StructType.fromDDL(schemaString) - val schema = CharVarcharUtils.failIfHasCharVarchar(rawSchema).asInstanceOf[StructType] - this.userSpecifiedSchema = Option(schema) - this + schema(StructType.fromDDL(schemaString)) } /** @@ -413,7 +412,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
  • `STOP_AT_DELIMITER`: If unescaped quotes are found in the input, consider the value * as an unquoted value. This will make the parser accumulate all characters until the * delimiter or a line ending is found in the input.
  • - *
  • `STOP_AT_DELIMITER`: If unescaped quotes are found in the input, the content parsed + *
  • `SKIP_VALUE`: If unescaped quotes are found in the input, the content parsed * for the given value will be skipped and the value set in nullValue will be produced * instead.
  • *
  • `RAISE_ERROR`: If unescaped quotes are found in the input, a TextParsingException diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index 0b5951ec2ac97..d8b630d5354e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -76,7 +76,11 @@ trait QueryExecutionListener { class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) extends Logging { - private val listenerBus = new ExecutionListenerBus(session) + private val listenerBus = new ExecutionListenerBus(session.sessionUUID) + session.sparkContext.listenerBus.addToSharedQueue(listenerBus) + session.sparkContext.cleaner.foreach { cleaner => + cleaner.registerSparkListenerForCleanup(this, listenerBus) + } if (loadExtensions) { val conf = session.sparkContext.conf @@ -124,11 +128,9 @@ class ExecutionListenerManager private[sql](session: SparkSession, loadExtension } } -private[sql] class ExecutionListenerBus(session: SparkSession) +private[sql] class ExecutionListenerBus(sessionUUID: String) extends SparkListener with ListenerBus[QueryExecutionListener, SparkListenerSQLExecutionEnd] { - session.sparkContext.listenerBus.addToSharedQueue(this) - override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case e: SparkListenerSQLExecutionEnd => postToAll(e) case _ => @@ -158,6 +160,6 @@ private[sql] class ExecutionListenerBus(session: SparkSession) private def shouldReport(e: SparkListenerSQLExecutionEnd): Boolean = { // Only catch SQL execution with a name, and triggered by the same spark session that this // listener manager belongs. - e.executionName.isDefined && e.qe.sparkSession.eq(this.session) + e.executionName.isDefined && e.qe.sparkSession.sessionUUID == sessionUUID } } diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql index eabbd0a932253..936da959efabf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-select.sql @@ -128,3 +128,13 @@ WHERE NOT EXISTS (SELECT (SELECT max(t2b) ON t2a = t1a WHERE t2c = t3c) AND t3a = t1a); + +-- SPARK-34876: Non-nullable aggregates should not return NULL in a correlated subquery +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT sort_array(collect_set(t2d)) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 93b0cc3fe97e1..b0d650ff062e9 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -539,7 +539,7 @@ EXPLAIN EXTENDED DESC t struct -- !query output == Parsed Logical Plan == -'DescribeRelation false +'DescribeRelation false, [col_name#x, data_type#x, comment#x] +- 'UnresolvedTableOrView [t], DESCRIBE TABLE, true == Analyzed Logical Plan == @@ -571,7 +571,7 @@ struct -- !query output == Physical Plan == Execute DescribeTableCommand - +- DescribeTableCommand `default`.`t`, Map(c -> Us, d -> 2), false + +- DescribeTableCommand `default`.`t`, [c=Us, d=2], false -- !query diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 184b8daf9d28e..68aad89e407a2 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 11 +-- Number of queries: 12 -- !query @@ -196,3 +196,29 @@ val1d NULL val1e 10 val1e 10 val1e 10 + + +-- !query +SELECT t1a, + (SELECT count(t2d) FROM t2 WHERE t2a = t1a) count_t2, + (SELECT count_if(t2d > 0) FROM t2 WHERE t2a = t1a) count_if_t2, + (SELECT approx_count_distinct(t2d) FROM t2 WHERE t2a = t1a) approx_count_distinct_t2, + (SELECT collect_list(t2d) FROM t2 WHERE t2a = t1a) collect_list_t2, + (SELECT sort_array(collect_set(t2d)) FROM t2 WHERE t2a = t1a) collect_set_t2, + (SELECT hex(count_min_sketch(t2d, 0.5d, 0.5d, 1)) FROM t2 WHERE t2a = t1a) collect_set_t2 +FROM t1 +-- !query schema +struct,collect_set_t2:array,collect_set_t2:string> +-- !query output +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1a 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1b 6 6 3 [19,119,319,19,19,19] [19,119,319] 0000000100000000000000060000000100000004000000005D8D6AB90000000000000000000000000000000400000000000000010000000000000001 +val1c 2 2 2 [219,19] [19,219] 0000000100000000000000020000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000001 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1d 0 0 0 [] [] 0000000100000000000000000000000100000004000000005D8D6AB90000000000000000000000000000000000000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 +val1e 1 1 1 [19] [19] 0000000100000000000000010000000100000004000000005D8D6AB90000000000000000000000000000000100000000000000000000000000000000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out index 054ee00ecc2ae..43506b49a6683 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -100,6 +100,14 @@ WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) FROM t2 WHERE t2.k = t1.k) -- !query schema -struct +struct<> -- !query output -two +org.apache.spark.sql.AnalysisException +Correlated column is not allowed in predicate (CAST(udf(cast(k as string)) AS STRING) = CAST(udf(cast(outer(k#x) as string)) AS STRING)): +Aggregate [cast(udf(cast(max(cast(udf(cast(v#x as string)) as int)) as string)) as int) AS CAST(udf(cast(max(cast(udf(cast(v as string)) as int)) as string)) AS INT)#x] ++- Filter (cast(udf(cast(k#x as string)) as string) = cast(udf(cast(outer(k#x) as string)) as string)) + +- SubqueryAlias t2 + +- View (`t2`, [k#x,v#x]) + +- Project [k#x, v#x] + +- SubqueryAlias t2 + +- LocalRelation [k#x, v#x] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt index 25da173c8ecde..cfdd41a08844c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q73.sf100/explain.txt @@ -117,7 +117,7 @@ Input [5]: [ss_customer_sk#2, ss_hdemo_sk#3, ss_store_sk#4, ss_ticket_number#5, Output [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] Batched: true Location [not included in comparison]/{warehouse_dir}/household_demographics] -PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000),EqualTo(hd_buy_potential,Unknown)), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_vehicle_count), Or(EqualTo(hd_buy_potential,>10000 ),EqualTo(hd_buy_potential,Unknown )), GreaterThan(hd_vehicle_count,0), IsNotNull(hd_demo_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] @@ -125,7 +125,7 @@ Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_coun (20) Filter [codegen id : 3] Input [4]: [hd_demo_sk#13, hd_buy_potential#14, hd_dep_count#15, hd_vehicle_count#16] -Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000) OR (hd_buy_potential#14 = Unknown))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) +Condition : ((((isnotnull(hd_vehicle_count#16) AND ((hd_buy_potential#14 = >10000 ) OR (hd_buy_potential#14 = Unknown ))) AND (hd_vehicle_count#16 > 0)) AND (CASE WHEN (hd_vehicle_count#16 > 0) THEN (cast(hd_dep_count#15 as double) / cast(hd_vehicle_count#16 as double)) ELSE null END > 1.0)) AND isnotnull(hd_demo_sk#13)) (21) Project [codegen id : 3] Output [1]: [hd_demo_sk#13] @@ -153,7 +153,7 @@ Results [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] (26) Exchange Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] -Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), true, [id=#20] +Arguments: hashpartitioning(ss_ticket_number#5, ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#20] (27) HashAggregate [codegen id : 5] Input [3]: [ss_ticket_number#5, ss_customer_sk#2, count#19] @@ -195,7 +195,7 @@ Input [8]: [ss_ticket_number#5, ss_customer_sk#2, cnt#22, c_customer_sk#24, c_sa (35) Exchange Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] -Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), true, [id=#29] +Arguments: rangepartitioning(cnt#22 DESC NULLS LAST, 5), ENSURE_REQUIREMENTS, [id=#29] (36) Sort [codegen id : 7] Input [6]: [c_last_name#27, c_first_name#26, c_salutation#25, c_preferred_cust_flag#28, ss_ticket_number#5, cnt#22] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt index b4dd8173664b6..f4dc8d6c5d6df 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt index e9b94a6b4651c..2207b0fee23ce 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 327e7db702faa..3dc7034581000 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 (29) - : +- * BroadcastHashJoin Inner BuildRight (28) + :- * Project (28) + : +- * BroadcastHashJoin Inner BuildRight (27) : :- * Project (22) : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * Project (15) @@ -27,16 +27,16 @@ : : +- * Project (19) : : +- * Filter (18) : : +- * ColumnarToRow (17) - : : +- Scan parquet default.customer_address (16) - : +- BroadcastExchange (27) - : +- * Project (26) - : +- * Filter (25) - : +- * ColumnarToRow (24) - : +- Scan parquet default.date_dim (23) + : : +- Scan parquet default.date_dim (16) + : +- BroadcastExchange (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.store (23) +- BroadcastExchange (33) - +- * Filter (32) - +- * ColumnarToRow (31) - +- Scan parquet default.store (30) + +- * Project (32) + +- * Filter (31) + +- * ColumnarToRow (30) + +- Scan parquet default.customer_address (29) (1) Scan parquet default.store_sales @@ -57,7 +57,7 @@ Condition : ((((((isnotnull(ss_store_sk#5) AND isnotnull(ss_addr_sk#4)) AND isno Output [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree)),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree)))] +PushedFilters: [IsNotNull(cd_demo_sk), Or(Or(And(EqualTo(cd_marital_status,M),EqualTo(cd_education_status,Advanced Degree )),And(EqualTo(cd_marital_status,S),EqualTo(cd_education_status,College ))),And(EqualTo(cd_marital_status,W),EqualTo(cd_education_status,2 yr Degree )))] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -65,7 +65,7 @@ Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] (6) Filter [codegen id : 1] Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] -Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)))) +Condition : (isnotnull(cd_demo_sk#11) AND ((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) OR ((cd_marital_status#12 = S) AND (cd_education_status#13 = College ))) OR ((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )))) (7) BroadcastExchange Input [3]: [cd_demo_sk#11, cd_marital_status#12, cd_education_status#13] @@ -74,7 +74,7 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (8) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_cdemo_sk#2] Right keys [1]: [cd_demo_sk#11] -Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) +Join condition: ((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) OR ((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00))) OR ((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00))) (9) Project [codegen id : 6] Output [11]: [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] @@ -101,100 +101,100 @@ Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint) (14) BroadcastHashJoin [codegen id : 6] Left keys [1]: [ss_hdemo_sk#3] Right keys [1]: [hd_demo_sk#15] -Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree)) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College)) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree)) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) +Join condition: (((((((cd_marital_status#12 = M) AND (cd_education_status#13 = Advanced Degree )) AND (ss_sales_price#7 >= 100.00)) AND (ss_sales_price#7 <= 150.00)) AND (hd_dep_count#16 = 3)) OR (((((cd_marital_status#12 = S) AND (cd_education_status#13 = College )) AND (ss_sales_price#7 >= 50.00)) AND (ss_sales_price#7 <= 100.00)) AND (hd_dep_count#16 = 1))) OR (((((cd_marital_status#12 = W) AND (cd_education_status#13 = 2 yr Degree )) AND (ss_sales_price#7 >= 150.00)) AND (ss_sales_price#7 <= 200.00)) AND (hd_dep_count#16 = 1))) (15) Project [codegen id : 6] 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.customer_address -Output [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +(16) Scan parquet default.date_dim +Output [2]: [d_date_sk#18, d_year#19] 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 +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)] +ReadSchema: struct (17) ColumnarToRow [codegen id : 3] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Input [2]: [d_date_sk#18, d_year#19] (18) Filter [codegen id : 3] -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))) +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)) (19) Project [codegen id : 3] -Output [2]: [ca_address_sk#18, ca_state#19] -Input [3]: [ca_address_sk#18, ca_state#19, ca_country#20] +Output [1]: [d_date_sk#18] +Input [2]: [d_date_sk#18, d_year#19] (20) BroadcastExchange -Input [2]: [ca_address_sk#18, ca_state#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#21] +Input [1]: [d_date_sk#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#20] (21) BroadcastHashJoin [codegen id : 6] -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))) +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#18] +Join condition: None (22) Project [codegen id : 6] -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] +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] -(23) Scan parquet default.date_dim -Output [2]: [d_date_sk#22, d_year#23] +(23) Scan parquet default.store +Output [1]: [s_store_sk#21] 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}/store] +PushedFilters: [IsNotNull(s_store_sk)] +ReadSchema: struct (24) ColumnarToRow [codegen id : 4] -Input [2]: [d_date_sk#22, d_year#23] +Input [1]: [s_store_sk#21] (25) Filter [codegen id : 4] -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) Project [codegen id : 4] -Output [1]: [d_date_sk#22] -Input [2]: [d_date_sk#22, d_year#23] +Input [1]: [s_store_sk#21] +Condition : isnotnull(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] +(26) BroadcastExchange +Input [1]: [s_store_sk#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#22] -(28) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#22] +(27) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_store_sk#5] +Right keys [1]: [s_store_sk#21] Join condition: None -(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] +(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] -(30) Scan parquet default.store -Output [1]: [s_store_sk#25] +(29) Scan parquet default.customer_address +Output [3]: [ca_address_sk#23, ca_state#24, ca_country#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}/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] -(31) ColumnarToRow [codegen id : 5] -Input [1]: [s_store_sk#25] +(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))) -(32) Filter [codegen id : 5] -Input [1]: [s_store_sk#25] -Condition : isnotnull(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] (33) BroadcastExchange -Input [1]: [s_store_sk#25] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#26] +Input [2]: [ca_address_sk#23, ca_state#24] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#26] (34) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [ss_store_sk#5] -Right keys [1]: [s_store_sk#25] -Join condition: None +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))) (35) Project [codegen id : 6] Output [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] -Input [5]: [ss_store_sk#5, ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9, s_store_sk#25] +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] (36) HashAggregate [codegen id : 6] Input [3]: [ss_quantity#6, ss_ext_sales_price#8, ss_ext_wholesale_cost#9] 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 45d6c8f3b0bae..b457788dbd0b2 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_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] + 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] 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 [ca_address_sk,ca_state] - Filter [ca_country,ca_address_sk,ca_state] + Project [d_date_sk] + Filter [d_year,d_date_sk] ColumnarToRow InputAdapter - Scan parquet default.customer_address [ca_address_sk,ca_state,ca_country] + Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter BroadcastExchange #5 WholeStageCodegen (4) - Project [d_date_sk] - Filter [d_year,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_year] + Filter [s_store_sk] + ColumnarToRow + InputAdapter + Scan parquet default.store [s_store_sk] InputAdapter BroadcastExchange #6 WholeStageCodegen (5) - Filter [s_store_sk] - ColumnarToRow - InputAdapter - Scan parquet default.store [s_store_sk] + 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] 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 12e95ba50cd0d..f7927aad003a2 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,261 +34,272 @@ TakeOrderedAndProject (53) +- * Sort (46) +- Exchange (45) +- * Project (44) - +- * 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) + +- * 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) (1) 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] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [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] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [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] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,F), EqualTo(cd_education_status,Unknown ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = F)) AND (cd_education_status#12 = Unknown)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = F)) AND (cd_education_status#13 = Unknown )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [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, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), EqualTo(d_year,1998), IsNotNull(d_date_sk)] ReadSchema: struct (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 1998)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 1998)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -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] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, 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_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (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), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (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] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true 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 : 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] +(27) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(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)) +(28) Filter [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (1,6,8,9,12,2) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(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] +(29) Project [codegen id : 7] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(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] - -(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 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#28] +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 -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#28] -Condition : isnotnull(cd_demo_sk#28) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (MS,IN,ND,OK,NM,VA) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#28] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#28] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_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] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] + +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] 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 +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#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(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)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(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] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -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] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -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] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -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 +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] 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#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] +Output [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (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#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] +Input [11]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Arguments: [List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 0), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, ca_state#30, null, 1), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, ca_country#31, null, null, 3), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#20, null, null, null, 7), List(cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, null, null, null, null, 15)], [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] (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] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [partial_avg(cast(cs_quantity#5 as decimal(12,2))), partial_avg(cast(cs_list_price#6 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#8 as decimal(12,2))), partial_avg(cast(cs_sales_price#7 as decimal(12,2))), partial_avg(cast(cs_net_profit#9 as decimal(12,2))), partial_avg(cast(c_birth_year#26 as decimal(12,2))), partial_avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [14]: [sum#41, count#42, sum#43, count#44, sum#45, count#46, sum#47, count#48, sum#49, count#50, sum#51, count#52, sum#53, count#54] -Results [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] +Input [12]: [cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, c_birth_year#27, i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [partial_avg(cast(cs_quantity#4 as decimal(12,2))), partial_avg(cast(cs_list_price#5 as decimal(12,2))), partial_avg(cast(cs_coupon_amt#7 as decimal(12,2))), partial_avg(cast(cs_sales_price#6 as decimal(12,2))), partial_avg(cast(cs_net_profit#8 as decimal(12,2))), partial_avg(cast(c_birth_year#27 as decimal(12,2))), partial_avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [14]: [sum#42, count#43, sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55] +Results [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] (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), ENSURE_REQUIREMENTS, [id=#69] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Arguments: hashpartitioning(i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, 5), ENSURE_REQUIREMENTS, [id=#70] (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] -Keys [5]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, spark_grouping_id#40] -Functions [7]: [avg(cast(cs_quantity#5 as decimal(12,2))), avg(cast(cs_list_price#6 as decimal(12,2))), avg(cast(cs_coupon_amt#8 as decimal(12,2))), avg(cast(cs_sales_price#7 as decimal(12,2))), avg(cast(cs_net_profit#9 as decimal(12,2))), avg(cast(c_birth_year#26 as decimal(12,2))), avg(cast(cd_dep_count#13 as decimal(12,2)))] -Aggregate Attributes [7]: [avg(cast(cs_quantity#5 as decimal(12,2)))#70, avg(cast(cs_list_price#6 as decimal(12,2)))#71, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72, avg(cast(cs_sales_price#7 as decimal(12,2)))#73, avg(cast(cs_net_profit#9 as decimal(12,2)))#74, avg(cast(c_birth_year#26 as decimal(12,2)))#75, avg(cast(cd_dep_count#13 as decimal(12,2)))#76] -Results [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, avg(cast(cs_quantity#5 as decimal(12,2)))#70 AS agg1#77, avg(cast(cs_list_price#6 as decimal(12,2)))#71 AS agg2#78, avg(cast(cs_coupon_amt#8 as decimal(12,2)))#72 AS agg3#79, avg(cast(cs_sales_price#7 as decimal(12,2)))#73 AS agg4#80, avg(cast(cs_net_profit#9 as decimal(12,2)))#74 AS agg5#81, avg(cast(c_birth_year#26 as decimal(12,2)))#75 AS agg6#82, avg(cast(cd_dep_count#13 as decimal(12,2)))#76 AS agg7#83] +Input [19]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41, sum#56, count#57, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69] +Keys [5]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, spark_grouping_id#41] +Functions [7]: [avg(cast(cs_quantity#4 as decimal(12,2))), avg(cast(cs_list_price#5 as decimal(12,2))), avg(cast(cs_coupon_amt#7 as decimal(12,2))), avg(cast(cs_sales_price#6 as decimal(12,2))), avg(cast(cs_net_profit#8 as decimal(12,2))), avg(cast(c_birth_year#27 as decimal(12,2))), avg(cast(cd_dep_count#14 as decimal(12,2)))] +Aggregate Attributes [7]: [avg(cast(cs_quantity#4 as decimal(12,2)))#71, avg(cast(cs_list_price#5 as decimal(12,2)))#72, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73, avg(cast(cs_sales_price#6 as decimal(12,2)))#74, avg(cast(cs_net_profit#8 as decimal(12,2)))#75, avg(cast(c_birth_year#27 as decimal(12,2)))#76, avg(cast(cd_dep_count#14 as decimal(12,2)))#77] +Results [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, avg(cast(cs_quantity#4 as decimal(12,2)))#71 AS agg1#78, avg(cast(cs_list_price#5 as decimal(12,2)))#72 AS agg2#79, avg(cast(cs_coupon_amt#7 as decimal(12,2)))#73 AS agg3#80, avg(cast(cs_sales_price#6 as decimal(12,2)))#74 AS agg4#81, avg(cast(cs_net_profit#8 as decimal(12,2)))#75 AS agg5#82, avg(cast(c_birth_year#27 as decimal(12,2)))#76 AS agg6#83, avg(cast(cd_dep_count#14 as decimal(12,2)))#77 AS agg7#84] (53) TakeOrderedAndProject -Input [11]: [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] -Arguments: 100, [ca_country#37 ASC NULLS FIRST, ca_state#38 ASC NULLS FIRST, ca_county#39 ASC NULLS FIRST, i_item_id#36 ASC NULLS FIRST], [i_item_id#36, ca_country#37, ca_state#38, ca_county#39, agg1#77, agg2#78, agg3#79, agg4#80, agg5#81, agg6#82, agg7#83] +Input [11]: [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] +Arguments: 100, [ca_country#38 ASC NULLS FIRST, ca_state#39 ASC NULLS FIRST, ca_county#40 ASC NULLS FIRST, i_item_id#37 ASC NULLS FIRST], [i_item_id#37, ca_country#38, ca_state#39, ca_county#40, agg1#78, agg2#79, agg3#80, agg4#81, agg5#82, agg6#83, agg7#84] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (54) + + +(54) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + 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 8069d43c3451a..8c76e7cab3310 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] - 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) + 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] 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 - 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 + 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 - 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 (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] 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 4627bc19f25f0..89dfa65b4aa37 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) - +- * 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] + +- * 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] 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 +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)) -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(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] -(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)) +(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] -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(9) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#6] +Join condition: None -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(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] -(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] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#13, d_year#14, d_moy#15] 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 +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)) -(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] +(14) Project [codegen id : 2] +Output [1]: [d_date_sk#13] +Input [3]: [d_date_sk#13, d_year#14, d_moy#15] -(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)) +(15) BroadcastExchange +Input [1]: [d_date_sk#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] -(9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +(16) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#13] Join condition: None -(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] +(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] -(11) Scan parquet default.store -Output [2]: [s_store_sk#10, s_zip#11] +(18) Scan parquet default.store +Output [2]: [s_store_sk#17, s_zip#18] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_zip), IsNotNull(s_store_sk)] ReadSchema: struct -(12) ColumnarToRow [codegen id : 2] -Input [2]: [s_store_sk#10, s_zip#11] +(19) ColumnarToRow [codegen id : 3] +Input [2]: [s_store_sk#17, s_zip#18] -(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)) +(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)) -(14) BroadcastExchange -Input [2]: [s_store_sk#10, s_zip#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(21) BroadcastExchange +Input [2]: [s_store_sk#17, s_zip#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#19] -(15) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_store_sk#8] -Right keys [1]: [s_store_sk#10] +(22) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#4] +Right keys [1]: [s_store_sk#17] Join condition: None -(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] +(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] -(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] +(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), ENSURE_REQUIREMENTS, [id=#20] -(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 +(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 -(19) Scan parquet default.customer -Output [2]: [c_customer_sk#14, c_current_addr_sk#15] +(26) Scan parquet default.customer +Output [2]: [c_customer_sk#21, c_current_addr_sk#22] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 5] -Input [2]: [c_customer_sk#14, c_current_addr_sk#15] +(27) ColumnarToRow [codegen id : 6] +Input [2]: [c_customer_sk#21, 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)) +(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)) -(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] +(29) Exchange +Input [2]: [c_customer_sk#21, c_current_addr_sk#22] +Arguments: hashpartitioning(c_current_addr_sk#22, 5), ENSURE_REQUIREMENTS, [id=#23] -(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 +(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 -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_zip#18] +(31) Scan parquet default.customer_address +Output [2]: [ca_address_sk#24, ca_zip#25] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow [codegen id : 7] -Input [2]: [ca_address_sk#17, ca_zip#18] +(32) ColumnarToRow [codegen id : 8] +Input [2]: [ca_address_sk#24, 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)) +(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)) -(27) Exchange -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: hashpartitioning(ca_address_sk#17, 5), ENSURE_REQUIREMENTS, [id=#19] +(34) Exchange +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: hashpartitioning(ca_address_sk#24, 5), ENSURE_REQUIREMENTS, [id=#26] -(28) Sort [codegen id : 8] -Input [2]: [ca_address_sk#17, ca_zip#18] -Arguments: [ca_address_sk#17 ASC NULLS FIRST], false, 0 +(35) Sort [codegen id : 9] +Input [2]: [ca_address_sk#24, ca_zip#25] +Arguments: [ca_address_sk#24 ASC NULLS FIRST], false, 0 -(29) SortMergeJoin [codegen id : 9] -Left keys [1]: [c_current_addr_sk#15] -Right keys [1]: [ca_address_sk#17] +(36) SortMergeJoin [codegen id : 10] +Left keys [1]: [c_current_addr_sk#22] +Right keys [1]: [ca_address_sk#24] Join condition: None -(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 +(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] -(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)) +(38) Exchange +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: hashpartitioning(c_customer_sk#21, 5), ENSURE_REQUIREMENTS, [id=#27] -(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] +(39) Sort [codegen id : 11] +Input [2]: [c_customer_sk#21, ca_zip#25] +Arguments: [c_customer_sk#21 ASC NULLS FIRST], false, 0 -(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 +(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)) (41) Project [codegen id : 12] -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] +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] (42) HashAggregate [codegen id : 12] -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))] +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))] Aggregate Attributes [1]: [sum#28] -Results [5]: [i_brand#23, i_brand_id#22, i_manufact_id#24, i_manufact#25, sum#29] +Results [5]: [i_brand#8, i_brand_id#7, i_manufact_id#9, i_manufact#10, sum#29] (43) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#30] (44) HashAggregate [codegen id : 13] -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] +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] (45) TakeOrderedAndProject -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] +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] 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 b6441c5fe72c1..05fa3f82e27df 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] - 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] + 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] InputAdapter BroadcastExchange #3 WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_moy,d_year,d_date_sk] + 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.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] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manufact_id,i_manufact,i_manager_id] InputAdapter BroadcastExchange #4 WholeStageCodegen (2) - 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] + 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 InputAdapter - 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] + Scan parquet default.store [s_store_sk,s_zip] InputAdapter - 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] + 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] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt index 2ef3660bc7ba4..a5b9623cbd96e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt index 1e8ab18f5e21a..965b68218eb0a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 093c4eed6cf11..351c7f0024dd0 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 @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) 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] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,pale ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -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_color#9) AND (i_color#9 = pale)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = pale )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [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] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(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), ENSURE_REQUIREMENTS, [id=#13] +(11) 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#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(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] +(12) 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#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -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)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(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), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -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 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, 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, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [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, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [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] -Input [17]: [ss_item_sk#1, 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, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(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), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(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] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -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, 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, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -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] -Keys [10]: [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] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [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] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(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), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(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] -Keys [10]: [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] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(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), 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] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [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] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(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_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(62) Exchange +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(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_customer_sk#2 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 3] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] 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 -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -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)) +(66) Filter [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) -(64) 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), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] -(65) Sort [codegen id : 5] -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 +(68) Sort [codegen id : 5] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -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] +(70) Project [codegen id : 6] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(68) Exchange -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] +(71) Exchange +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] -(69) Sort [codegen id : 7] -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 +(72) Sort [codegen id : 7] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(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] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] 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 -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -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) +(75) Filter [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) -(73) 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), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] -(74) Sort [codegen id : 9] -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 +(77) Sort [codegen id : 9] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] 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, 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] +(79) Project [codegen id : 10] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(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, 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] +(80) Exchange +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] -(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, 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 +(81) Sort [codegen id : 11] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] 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, 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] +(88) Project [codegen id : 14] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] -(86) Exchange -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] +(89) Exchange +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] -(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, 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 +(90) Sort [codegen id : 15] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -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, 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] -Keys [10]: [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] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [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] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(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), 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] -Keys [10]: [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] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] + +(99) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] + +(100) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] + +(101) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 7de562c5d59a1..10f874f8f5543 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,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + 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] 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,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] + 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] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - 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] + 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] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_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 [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - 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 (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - 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 (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 273950bed3546..97ee167a14b5a 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 @@ -1,567 +1,587 @@ == Physical Plan == -* Project (48) -+- * Filter (47) - +- * HashAggregate (46) - +- Exchange (45) - +- * HashAggregate (44) - +- * HashAggregate (43) - +- Exchange (42) - +- * HashAggregate (41) - +- * Project (40) - +- * SortMergeJoin Inner (39) - :- * Sort (33) - : +- Exchange (32) - : +- * Project (31) - : +- * BroadcastHashJoin Inner BuildRight (30) - : :- * Project (18) - : : +- * SortMergeJoin Inner (17) - : : :- * Sort (11) - : : : +- Exchange (10) - : : : +- * Project (9) - : : : +- * BroadcastHashJoin Inner BuildRight (8) - : : : :- * Filter (3) - : : : : +- * ColumnarToRow (2) - : : : : +- Scan parquet default.store_sales (1) - : : : +- BroadcastExchange (7) - : : : +- * Filter (6) - : : : +- * ColumnarToRow (5) - : : : +- Scan parquet default.item (4) - : : +- * Sort (16) - : : +- Exchange (15) - : : +- * Filter (14) - : : +- * ColumnarToRow (13) - : : +- Scan parquet default.customer (12) - : +- BroadcastExchange (29) - : +- * Project (28) - : +- * BroadcastHashJoin Inner BuildLeft (27) - : :- BroadcastExchange (23) - : : +- * Project (22) - : : +- * Filter (21) - : : +- * ColumnarToRow (20) - : : +- Scan parquet default.store (19) - : +- * Filter (26) - : +- * ColumnarToRow (25) - : +- Scan parquet default.customer_address (24) - +- * Sort (38) - +- Exchange (37) - +- * Filter (36) - +- * ColumnarToRow (35) - +- Scan parquet default.store_returns (34) +* Project (50) ++- * Filter (49) + +- * HashAggregate (48) + +- Exchange (47) + +- * HashAggregate (46) + +- * HashAggregate (45) + +- Exchange (44) + +- * HashAggregate (43) + +- * Project (42) + +- * SortMergeJoin Inner (41) + :- * Sort (34) + : +- Exchange (33) + : +- * Project (32) + : +- * BroadcastHashJoin Inner BuildRight (31) + : :- * Project (19) + : : +- * SortMergeJoin Inner (18) + : : :- * Sort (12) + : : : +- Exchange (11) + : : : +- * Project (10) + : : : +- * BroadcastHashJoin Inner BuildRight (9) + : : : :- * Project (4) + : : : : +- * Filter (3) + : : : : +- * ColumnarToRow (2) + : : : : +- Scan parquet default.store_sales (1) + : : : +- BroadcastExchange (8) + : : : +- * Filter (7) + : : : +- * ColumnarToRow (6) + : : : +- Scan parquet default.item (5) + : : +- * Sort (17) + : : +- Exchange (16) + : : +- * Filter (15) + : : +- * ColumnarToRow (14) + : : +- Scan parquet default.customer (13) + : +- BroadcastExchange (30) + : +- * Project (29) + : +- * BroadcastHashJoin Inner BuildLeft (28) + : :- BroadcastExchange (24) + : : +- * Project (23) + : : +- * Filter (22) + : : +- * ColumnarToRow (21) + : : +- Scan parquet default.store (20) + : +- * Filter (27) + : +- * ColumnarToRow (26) + : +- Scan parquet default.customer_address (25) + +- * Sort (40) + +- Exchange (39) + +- * Project (38) + +- * Filter (37) + +- * ColumnarToRow (36) + +- Scan parquet default.store_returns (35) (1) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Output [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] (3) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) -(4) 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] +(4) Project [codegen id : 2] +Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Input [6]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, ss_sold_date_sk#6] + +(5) Scan parquet default.item +Output [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon), IsNotNull(i_item_sk)] +PushedFilters: [IsNotNull(i_color), EqualTo(i_color,chiffon ), IsNotNull(i_item_sk)] ReadSchema: struct -(5) ColumnarToRow [codegen id : 1] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(6) ColumnarToRow [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(6) Filter [codegen id : 1] -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_color#9) AND (i_color#9 = chiffon)) AND isnotnull(i_item_sk#6)) +(7) Filter [codegen id : 1] +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Condition : ((isnotnull(i_color#10) AND (i_color#10 = chiffon )) AND isnotnull(i_item_sk#7)) -(7) BroadcastExchange -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#12] +(8) BroadcastExchange +Input [6]: [i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#13] -(8) BroadcastHashJoin [codegen id : 2] +(9) BroadcastHashJoin [codegen id : 2] Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +Right keys [1]: [i_item_sk#7] Join condition: None -(9) Project [codegen id : 2] -Output [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] -Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(10) Project [codegen id : 2] +Output [10]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Input [11]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_item_sk#7, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] -(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), ENSURE_REQUIREMENTS, [id=#13] +(11) 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#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] +Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#14] -(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] +(12) 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#8, i_size#9, i_color#10, i_units#11, i_manager_id#12] Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0 -(12) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(13) Scan parquet default.customer +Output [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] Batched: true Location [not included in comparison]/{warehouse_dir}/customer] PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_birth_country)] ReadSchema: struct -(13) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(14) ColumnarToRow [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(14) Filter [codegen id : 4] -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)) +(15) Filter [codegen id : 4] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Condition : (isnotnull(c_customer_sk#15) AND isnotnull(c_birth_country#18)) -(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), ENSURE_REQUIREMENTS, [id=#18] +(16) Exchange +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: hashpartitioning(c_customer_sk#15, 5), ENSURE_REQUIREMENTS, [id=#19] -(16) Sort [codegen id : 5] -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 +(17) Sort [codegen id : 5] +Input [4]: [c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] +Arguments: [c_customer_sk#15 ASC NULLS FIRST], false, 0 -(17) SortMergeJoin [codegen id : 8] +(18) SortMergeJoin [codegen id : 8] Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +Right keys [1]: [c_customer_sk#15] Join condition: None -(18) Project [codegen id : 8] -Output [12]: [ss_item_sk#1, 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, c_first_name#15, c_last_name#16, c_birth_country#17] -Input [14]: [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, c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(19) Project [codegen id : 8] +Output [12]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18] +Input [14]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_customer_sk#15, c_first_name#16, c_last_name#17, c_birth_country#18] -(19) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(20) Scan parquet default.store +Output [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(20) ColumnarToRow [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(21) ColumnarToRow [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(21) Filter [codegen id : 6] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(22) Filter [codegen id : 6] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] +Condition : (((isnotnull(s_market_id#22) AND (s_market_id#22 = 8)) AND isnotnull(s_store_sk#20)) AND isnotnull(s_zip#24)) -(22) Project [codegen id : 6] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(23) Project [codegen id : 6] +Output [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Input [5]: [s_store_sk#20, s_store_name#21, s_market_id#22, s_state#23, s_zip#24] -(23) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#24] +(24) BroadcastExchange +Input [4]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24] +Arguments: HashedRelationBroadcastMode(List(input[3, string, true]),false), [id=#25] -(24) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(25) Scan parquet default.customer_address +Output [3]: [ca_state#26, ca_zip#27, ca_country#28] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(25) ColumnarToRow -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(26) ColumnarToRow +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] -(26) Filter -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(27) Filter +Input [3]: [ca_state#26, ca_zip#27, ca_country#28] +Condition : (isnotnull(ca_country#28) AND isnotnull(ca_zip#27)) -(27) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [s_zip#23] -Right keys [1]: [ca_zip#26] +(28) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [s_zip#24] +Right keys [1]: [ca_zip#27] Join condition: None -(28) Project [codegen id : 7] -Output [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Input [7]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23, ca_state#25, ca_zip#26, ca_country#27] +(29) Project [codegen id : 7] +Output [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Input [7]: [s_store_sk#20, s_store_name#21, s_state#23, s_zip#24, ca_state#26, ca_zip#27, ca_country#28] -(29) BroadcastExchange -Input [5]: [s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] -Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#28] +(30) BroadcastExchange +Input [5]: [s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] +Arguments: HashedRelationBroadcastMode(List(input[0, int, true], upper(input[4, string, true])),false), [id=#29] -(30) BroadcastHashJoin [codegen id : 8] -Left keys [2]: [ss_store_sk#3, c_birth_country#17] -Right keys [2]: [s_store_sk#19, upper(ca_country#27)] +(31) BroadcastHashJoin [codegen id : 8] +Left keys [2]: [ss_store_sk#3, c_birth_country#18] +Right keys [2]: [s_store_sk#20, upper(ca_country#28)] Join condition: None -(31) Project [codegen id : 8] -Output [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] -Input [17]: [ss_item_sk#1, 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, c_first_name#15, c_last_name#16, c_birth_country#17, s_store_sk#19, s_store_name#20, s_state#22, ca_state#25, ca_country#27] +(32) Project [codegen id : 8] +Output [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Input [17]: [ss_item_sk#1, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, c_birth_country#18, s_store_sk#20, s_store_name#21, s_state#23, ca_state#26, ca_country#28] -(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), ENSURE_REQUIREMENTS, [id=#29] +(33) Exchange +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] +Arguments: hashpartitioning(cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint), 5), ENSURE_REQUIREMENTS, [id=#30] -(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] +(34) Sort [codegen id : 9] +Input [13]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26] Arguments: [cast(ss_ticket_number#4 as bigint) ASC NULLS FIRST, cast(ss_item_sk#1 as bigint) ASC NULLS FIRST], false, 0 -(34) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(35) Scan parquet default.store_returns +Output [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(35) ColumnarToRow [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(36) ColumnarToRow [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] + +(37) Filter [codegen id : 10] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] +Condition : (isnotnull(sr_ticket_number#32) AND isnotnull(sr_item_sk#31)) -(36) Filter [codegen id : 10] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(38) Project [codegen id : 10] +Output [2]: [sr_item_sk#31, sr_ticket_number#32] +Input [3]: [sr_item_sk#31, sr_ticket_number#32, sr_returned_date_sk#33] -(37) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#32] +(39) Exchange +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: hashpartitioning(sr_ticket_number#32, sr_item_sk#31, 5), ENSURE_REQUIREMENTS, [id=#34] -(38) Sort [codegen id : 11] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(40) Sort [codegen id : 11] +Input [2]: [sr_item_sk#31, sr_ticket_number#32] +Arguments: [sr_ticket_number#32 ASC NULLS FIRST, sr_item_sk#31 ASC NULLS FIRST], false, 0 -(39) SortMergeJoin [codegen id : 12] +(41) SortMergeJoin [codegen id : 12] Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] +Right keys [2]: [sr_ticket_number#32, sr_item_sk#31] Join condition: None -(40) Project [codegen id : 12] -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, 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, sr_item_sk#30, sr_ticket_number#31] +(42) Project [codegen id : 12] +Output [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Input [15]: [ss_item_sk#1, ss_ticket_number#4, ss_net_paid#5, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, s_store_name#21, s_state#23, ca_state#26, sr_item_sk#31, sr_ticket_number#32] -(41) HashAggregate [codegen id : 12] -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] -Keys [10]: [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] +(43) HashAggregate [codegen id : 12] +Input [11]: [ss_net_paid#5, s_store_name#21, s_state#23, i_current_price#8, i_size#9, i_color#10, i_units#11, i_manager_id#12, c_first_name#16, c_last_name#17, ca_state#26] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#33] -Results [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] +Aggregate Attributes [1]: [sum#35] +Results [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] -(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), ENSURE_REQUIREMENTS, [id=#35] +(44) Exchange +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, 5), ENSURE_REQUIREMENTS, [id=#37] -(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] -Keys [10]: [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] +(45) HashAggregate [codegen id : 13] +Input [11]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9, sum#36] +Keys [10]: [c_last_name#17, c_first_name#16, s_store_name#21, ca_state#26, s_state#23, i_color#10, i_current_price#8, i_manager_id#12, i_units#11, i_size#9] Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#36] -Results [4]: [c_last_name#16, c_first_name#15, s_store_name#20, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#36,17,2) AS netpaid#37] - -(44) HashAggregate [codegen id : 13] -Input [4]: [c_last_name#16, c_first_name#15, s_store_name#20, netpaid#37] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [partial_sum(netpaid#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum#40, isEmpty#41] - -(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), 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] -Keys [3]: [c_last_name#16, c_first_name#15, s_store_name#20] -Functions [1]: [sum(netpaid#37)] -Aggregate Attributes [1]: [sum(netpaid#37)#43] -Results [5]: [c_last_name#16, c_first_name#15, s_store_name#20, sum(netpaid#37)#43 AS paid#44, sum(netpaid#37)#43 AS sum(netpaid#37)#45] - -(47) Filter [codegen id : 14] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] -Condition : (isnotnull(sum(netpaid#37)#45) AND (cast(sum(netpaid#37)#45 as decimal(33,8)) > cast(Subquery scalar-subquery#46, [id=#47] as decimal(33,8)))) - -(48) Project [codegen id : 14] -Output [4]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44] -Input [5]: [c_last_name#16, c_first_name#15, s_store_name#20, paid#44, sum(netpaid#37)#45] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#38] +Results [4]: [c_last_name#17, c_first_name#16, s_store_name#21, MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#38,17,2) AS netpaid#39] + +(46) HashAggregate [codegen id : 13] +Input [4]: [c_last_name#17, c_first_name#16, s_store_name#21, netpaid#39] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [partial_sum(netpaid#39)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] + +(47) Exchange +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Arguments: hashpartitioning(c_last_name#17, c_first_name#16, s_store_name#21, 5), ENSURE_REQUIREMENTS, [id=#44] + +(48) HashAggregate [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum#42, isEmpty#43] +Keys [3]: [c_last_name#17, c_first_name#16, s_store_name#21] +Functions [1]: [sum(netpaid#39)] +Aggregate Attributes [1]: [sum(netpaid#39)#45] +Results [5]: [c_last_name#17, c_first_name#16, s_store_name#21, sum(netpaid#39)#45 AS paid#46, sum(netpaid#39)#45 AS sum(netpaid#39)#47] + +(49) Filter [codegen id : 14] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] +Condition : (isnotnull(sum(netpaid#39)#47) AND (cast(sum(netpaid#39)#47 as decimal(33,8)) > cast(Subquery scalar-subquery#48, [id=#49] as decimal(33,8)))) + +(50) Project [codegen id : 14] +Output [4]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46] +Input [5]: [c_last_name#17, c_first_name#16, s_store_name#21, paid#46, sum(netpaid#39)#47] ===== Subqueries ===== -Subquery:1 Hosting operator id = 47 Hosting Expression = Subquery scalar-subquery#46, [id=#47] -* HashAggregate (100) -+- Exchange (99) - +- * HashAggregate (98) - +- * HashAggregate (97) - +- Exchange (96) - +- * HashAggregate (95) - +- * Project (94) - +- * SortMergeJoin Inner (93) - :- * Sort (87) - : +- Exchange (86) - : +- * Project (85) - : +- * SortMergeJoin Inner (84) - : :- * Sort (78) - : : +- Exchange (77) - : : +- * Project (76) - : : +- * SortMergeJoin Inner (75) - : : :- * Sort (69) - : : : +- Exchange (68) - : : : +- * Project (67) - : : : +- * SortMergeJoin Inner (66) - : : : :- * Sort (60) - : : : : +- Exchange (59) - : : : : +- * Project (58) - : : : : +- * BroadcastHashJoin Inner BuildRight (57) - : : : : :- * Filter (51) - : : : : : +- * ColumnarToRow (50) - : : : : : +- Scan parquet default.store_sales (49) - : : : : +- BroadcastExchange (56) - : : : : +- * Project (55) - : : : : +- * Filter (54) - : : : : +- * ColumnarToRow (53) - : : : : +- Scan parquet default.store (52) - : : : +- * Sort (65) - : : : +- Exchange (64) - : : : +- * Filter (63) - : : : +- * ColumnarToRow (62) - : : : +- Scan parquet default.customer (61) - : : +- * Sort (74) - : : +- Exchange (73) - : : +- * Filter (72) - : : +- * ColumnarToRow (71) - : : +- Scan parquet default.item (70) - : +- * Sort (83) - : +- Exchange (82) - : +- * Filter (81) - : +- * ColumnarToRow (80) - : +- Scan parquet default.customer_address (79) - +- * Sort (92) - +- Exchange (91) - +- * Filter (90) - +- * ColumnarToRow (89) - +- Scan parquet default.store_returns (88) - - -(49) Scan parquet default.store_sales -Output [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +Subquery:1 Hosting operator id = 49 Hosting Expression = Subquery scalar-subquery#48, [id=#49] +* HashAggregate (104) ++- Exchange (103) + +- * HashAggregate (102) + +- * HashAggregate (101) + +- Exchange (100) + +- * HashAggregate (99) + +- * Project (98) + +- * SortMergeJoin Inner (97) + :- * Sort (90) + : +- Exchange (89) + : +- * Project (88) + : +- * SortMergeJoin Inner (87) + : :- * Sort (81) + : : +- Exchange (80) + : : +- * Project (79) + : : +- * SortMergeJoin Inner (78) + : : :- * Sort (72) + : : : +- Exchange (71) + : : : +- * Project (70) + : : : +- * SortMergeJoin Inner (69) + : : : :- * Sort (63) + : : : : +- Exchange (62) + : : : : +- * Project (61) + : : : : +- * BroadcastHashJoin Inner BuildRight (60) + : : : : :- * Project (54) + : : : : : +- * Filter (53) + : : : : : +- * ColumnarToRow (52) + : : : : : +- Scan parquet default.store_sales (51) + : : : : +- BroadcastExchange (59) + : : : : +- * Project (58) + : : : : +- * Filter (57) + : : : : +- * ColumnarToRow (56) + : : : : +- Scan parquet default.store (55) + : : : +- * Sort (68) + : : : +- Exchange (67) + : : : +- * Filter (66) + : : : +- * ColumnarToRow (65) + : : : +- Scan parquet default.item (64) + : : +- * Sort (77) + : : +- Exchange (76) + : : +- * Filter (75) + : : +- * ColumnarToRow (74) + : : +- Scan parquet default.customer (73) + : +- * Sort (86) + : +- Exchange (85) + : +- * Filter (84) + : +- * ColumnarToRow (83) + : +- Scan parquet default.customer_address (82) + +- * Sort (96) + +- Exchange (95) + +- * Project (94) + +- * Filter (93) + +- * ColumnarToRow (92) + +- Scan parquet default.store_returns (91) + + +(51) Scan parquet default.store_sales +Output [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk), IsNotNull(ss_store_sk), IsNotNull(ss_customer_sk)] ReadSchema: struct -(50) ColumnarToRow [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] +(52) ColumnarToRow [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] -(51) Filter [codegen id : 2] -Input [5]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5] -Condition : (((isnotnull(ss_ticket_number#4) AND isnotnull(ss_item_sk#1)) AND isnotnull(ss_store_sk#3)) AND isnotnull(ss_customer_sk#2)) +(53) Filter [codegen id : 2] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] +Condition : (((isnotnull(ss_ticket_number#53) AND isnotnull(ss_item_sk#50)) AND isnotnull(ss_store_sk#52)) AND isnotnull(ss_customer_sk#51)) -(52) Scan parquet default.store -Output [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(54) Project [codegen id : 2] +Output [5]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54] +Input [6]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, ss_sold_date_sk#55] + +(55) Scan parquet default.store +Output [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_market_id), EqualTo(s_market_id,8), IsNotNull(s_store_sk), IsNotNull(s_zip)] ReadSchema: struct -(53) ColumnarToRow [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(56) ColumnarToRow [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(54) Filter [codegen id : 1] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] -Condition : (((isnotnull(s_market_id#21) AND (s_market_id#21 = 8)) AND isnotnull(s_store_sk#19)) AND isnotnull(s_zip#23)) +(57) Filter [codegen id : 1] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] +Condition : (((isnotnull(s_market_id#58) AND (s_market_id#58 = 8)) AND isnotnull(s_store_sk#56)) AND isnotnull(s_zip#60)) -(55) Project [codegen id : 1] -Output [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Input [5]: [s_store_sk#19, s_store_name#20, s_market_id#21, s_state#22, s_zip#23] +(58) Project [codegen id : 1] +Output [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Input [5]: [s_store_sk#56, s_store_name#57, s_market_id#58, s_state#59, s_zip#60] -(56) BroadcastExchange -Input [4]: [s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#48] +(59) BroadcastExchange +Input [4]: [s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#61] -(57) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_store_sk#3] -Right keys [1]: [s_store_sk#19] +(60) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [ss_store_sk#52] +Right keys [1]: [s_store_sk#56] Join condition: None -(58) Project [codegen id : 2] -Output [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] -Input [9]: [ss_item_sk#1, ss_customer_sk#2, ss_store_sk#3, ss_ticket_number#4, ss_net_paid#5, s_store_sk#19, s_store_name#20, s_state#22, s_zip#23] +(61) Project [codegen id : 2] +Output [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Input [9]: [ss_item_sk#50, ss_customer_sk#51, ss_store_sk#52, ss_ticket_number#53, ss_net_paid#54, s_store_sk#56, s_store_name#57, s_state#59, s_zip#60] -(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_customer_sk#2, 5), ENSURE_REQUIREMENTS, [id=#49] +(62) Exchange +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: hashpartitioning(ss_item_sk#50, 5), ENSURE_REQUIREMENTS, [id=#62] -(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_customer_sk#2 ASC NULLS FIRST], false, 0 +(63) Sort [codegen id : 3] +Input [7]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60] +Arguments: [ss_item_sk#50 ASC NULLS FIRST], false, 0 -(61) Scan parquet default.customer -Output [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(64) Scan parquet default.item +Output [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] 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 -(62) ColumnarToRow [codegen id : 4] -Input [4]: [c_customer_sk#14, c_first_name#15, c_last_name#16, c_birth_country#17] +(65) ColumnarToRow [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(63) Filter [codegen id : 4] -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)) +(66) Filter [codegen id : 4] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Condition : isnotnull(i_item_sk#63) -(64) 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), ENSURE_REQUIREMENTS, [id=#50] +(67) Exchange +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(i_item_sk#63, 5), ENSURE_REQUIREMENTS, [id=#69] -(65) Sort [codegen id : 5] -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 +(68) Sort [codegen id : 5] +Input [6]: [i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [i_item_sk#63 ASC NULLS FIRST], false, 0 -(66) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_customer_sk#2] -Right keys [1]: [c_customer_sk#14] +(69) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#50] +Right keys [1]: [i_item_sk#63] Join condition: None -(67) Project [codegen id : 6] -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] +(70) Project [codegen id : 6] +Output [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Input [13]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_item_sk#63, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] -(68) Exchange -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] +(71) Exchange +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [id=#70] -(69) Sort [codegen id : 7] -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 +(72) Sort [codegen id : 7] +Input [12]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68] +Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0 -(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] +(73) Scan parquet default.customer +Output [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] 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 -(71) ColumnarToRow [codegen id : 8] -Input [6]: [i_item_sk#6, i_current_price#7, i_size#8, i_color#9, i_units#10, i_manager_id#11] +(74) ColumnarToRow [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(72) Filter [codegen id : 8] -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) +(75) Filter [codegen id : 8] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Condition : (isnotnull(c_customer_sk#71) AND isnotnull(c_birth_country#74)) -(73) 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), ENSURE_REQUIREMENTS, [id=#52] +(76) Exchange +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_customer_sk#71, 5), ENSURE_REQUIREMENTS, [id=#75] -(74) Sort [codegen id : 9] -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 +(77) Sort [codegen id : 9] +Input [4]: [c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0 -(75) SortMergeJoin [codegen id : 10] -Left keys [1]: [ss_item_sk#1] -Right keys [1]: [i_item_sk#6] +(78) SortMergeJoin [codegen id : 10] +Left keys [1]: [ss_customer_sk#51] +Right keys [1]: [c_customer_sk#71] 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, 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] +(79) Project [codegen id : 10] +Output [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Input [16]: [ss_item_sk#50, ss_customer_sk#51, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_customer_sk#71, c_first_name#72, c_last_name#73, c_birth_country#74] -(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, 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] +(80) Exchange +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: hashpartitioning(c_birth_country#74, s_zip#60, 5), ENSURE_REQUIREMENTS, [id=#76] -(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, 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 +(81) Sort [codegen id : 11] +Input [14]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74] +Arguments: [c_birth_country#74 ASC NULLS FIRST, s_zip#60 ASC NULLS FIRST], false, 0 -(79) Scan parquet default.customer_address -Output [3]: [ca_state#25, ca_zip#26, ca_country#27] +(82) Scan parquet default.customer_address +Output [3]: [ca_state#77, ca_zip#78, ca_country#79] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_address] PushedFilters: [IsNotNull(ca_country), IsNotNull(ca_zip)] ReadSchema: struct -(80) ColumnarToRow [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] +(83) ColumnarToRow [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] -(81) Filter [codegen id : 12] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Condition : (isnotnull(ca_country#27) AND isnotnull(ca_zip#26)) +(84) Filter [codegen id : 12] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Condition : (isnotnull(ca_country#79) AND isnotnull(ca_zip#78)) -(82) Exchange -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: hashpartitioning(upper(ca_country#27), ca_zip#26, 5), ENSURE_REQUIREMENTS, [id=#54] +(85) Exchange +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: hashpartitioning(upper(ca_country#79), ca_zip#78, 5), ENSURE_REQUIREMENTS, [id=#80] -(83) Sort [codegen id : 13] -Input [3]: [ca_state#25, ca_zip#26, ca_country#27] -Arguments: [upper(ca_country#27) ASC NULLS FIRST, ca_zip#26 ASC NULLS FIRST], false, 0 +(86) Sort [codegen id : 13] +Input [3]: [ca_state#77, ca_zip#78, ca_country#79] +Arguments: [upper(ca_country#79) ASC NULLS FIRST, ca_zip#78 ASC NULLS FIRST], false, 0 -(84) SortMergeJoin [codegen id : 14] -Left keys [2]: [c_birth_country#17, s_zip#23] -Right keys [2]: [upper(ca_country#27), ca_zip#26] +(87) SortMergeJoin [codegen id : 14] +Left keys [2]: [c_birth_country#74, s_zip#60] +Right keys [2]: [upper(ca_country#79), ca_zip#78] 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, 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] +(88) Project [codegen id : 14] +Output [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [17]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, s_zip#60, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, c_birth_country#74, ca_state#77, ca_zip#78, ca_country#79] -(86) Exchange -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] +(89) Exchange +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: hashpartitioning(cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint), 5), ENSURE_REQUIREMENTS, [id=#81] -(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, 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 +(90) Sort [codegen id : 15] +Input [13]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Arguments: [cast(ss_ticket_number#53 as bigint) ASC NULLS FIRST, cast(ss_item_sk#50 as bigint) ASC NULLS FIRST], false, 0 -(88) Scan parquet default.store_returns -Output [2]: [sr_item_sk#30, sr_ticket_number#31] +(91) Scan parquet default.store_returns +Output [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(89) ColumnarToRow [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] +(92) ColumnarToRow [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(90) Filter [codegen id : 16] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Condition : (isnotnull(sr_ticket_number#31) AND isnotnull(sr_item_sk#30)) +(93) Filter [codegen id : 16] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] +Condition : (isnotnull(sr_ticket_number#83) AND isnotnull(sr_item_sk#82)) -(91) Exchange -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: hashpartitioning(sr_ticket_number#31, sr_item_sk#30, 5), ENSURE_REQUIREMENTS, [id=#56] +(94) Project [codegen id : 16] +Output [2]: [sr_item_sk#82, sr_ticket_number#83] +Input [3]: [sr_item_sk#82, sr_ticket_number#83, sr_returned_date_sk#84] -(92) Sort [codegen id : 17] -Input [2]: [sr_item_sk#30, sr_ticket_number#31] -Arguments: [sr_ticket_number#31 ASC NULLS FIRST, sr_item_sk#30 ASC NULLS FIRST], false, 0 +(95) Exchange +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: hashpartitioning(sr_ticket_number#83, sr_item_sk#82, 5), ENSURE_REQUIREMENTS, [id=#85] -(93) SortMergeJoin [codegen id : 18] -Left keys [2]: [cast(ss_ticket_number#4 as bigint), cast(ss_item_sk#1 as bigint)] -Right keys [2]: [sr_ticket_number#31, sr_item_sk#30] -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, 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] -Keys [10]: [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] -Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum#57] -Results [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] +(96) Sort [codegen id : 17] +Input [2]: [sr_item_sk#82, sr_ticket_number#83] +Arguments: [sr_ticket_number#83 ASC NULLS FIRST, sr_item_sk#82 ASC NULLS FIRST], false, 0 -(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), 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] -Keys [10]: [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] -Functions [1]: [sum(UnscaledValue(ss_net_paid#5))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#5))#60] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#5))#60,17,2) AS netpaid#37] +(97) SortMergeJoin [codegen id : 18] +Left keys [2]: [cast(ss_ticket_number#53 as bigint), cast(ss_item_sk#50 as bigint)] +Right keys [2]: [sr_ticket_number#83, sr_item_sk#82] +Join condition: None -(98) HashAggregate [codegen id : 19] -Input [1]: [netpaid#37] +(98) Project [codegen id : 18] +Output [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Input [15]: [ss_item_sk#50, ss_ticket_number#53, ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77, sr_item_sk#82, sr_ticket_number#83] + +(99) HashAggregate [codegen id : 18] +Input [11]: [ss_net_paid#54, s_store_name#57, s_state#59, i_current_price#64, i_size#65, i_color#66, i_units#67, i_manager_id#68, c_first_name#72, c_last_name#73, ca_state#77] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [partial_sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum#86] +Results [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] + +(100) Exchange +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Arguments: hashpartitioning(c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, 5), ENSURE_REQUIREMENTS, [id=#88] + +(101) HashAggregate [codegen id : 19] +Input [11]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65, sum#87] +Keys [10]: [c_last_name#73, c_first_name#72, s_store_name#57, ca_state#77, s_state#59, i_color#66, i_current_price#64, i_manager_id#68, i_units#67, i_size#65] +Functions [1]: [sum(UnscaledValue(ss_net_paid#54))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_paid#54))#89] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_paid#54))#89,17,2) AS netpaid#39] + +(102) HashAggregate [codegen id : 19] +Input [1]: [netpaid#39] Keys: [] -Functions [1]: [partial_avg(netpaid#37)] -Aggregate Attributes [2]: [sum#61, count#62] -Results [2]: [sum#63, count#64] +Functions [1]: [partial_avg(netpaid#39)] +Aggregate Attributes [2]: [sum#90, count#91] +Results [2]: [sum#92, count#93] -(99) Exchange -Input [2]: [sum#63, count#64] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#65] +(103) Exchange +Input [2]: [sum#92, count#93] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#94] -(100) HashAggregate [codegen id : 20] -Input [2]: [sum#63, count#64] +(104) HashAggregate [codegen id : 20] +Input [2]: [sum#92, count#93] Keys: [] -Functions [1]: [avg(netpaid#37)] -Aggregate Attributes [1]: [avg(netpaid#37)#66] -Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#37)#66)), DecimalType(24,8), true) AS (CAST(0.05 AS DECIMAL(21,6)) * CAST(avg(netpaid) AS DECIMAL(21,6)))#67] +Functions [1]: [avg(netpaid#39)] +Aggregate Attributes [1]: [avg(netpaid#39)#95] +Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#39)#95)), DecimalType(24,8), true) AS (0.05 * avg(netpaid))#96] 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 7de562c5d59a1..10f874f8f5543 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,c_first_name,c_last_name,i_current_price,i_size,i_color,i_units,i_manager_id,ca_state] + 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] 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,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] + 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] InputAdapter WholeStageCodegen (7) - Sort [ss_item_sk] + Sort [ss_customer_sk] InputAdapter - Exchange [ss_item_sk] #14 + Exchange [ss_customer_sk] #14 WholeStageCodegen (6) - 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] + 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] InputAdapter WholeStageCodegen (3) - Sort [ss_customer_sk] + Sort [ss_item_sk] InputAdapter - Exchange [ss_customer_sk] #15 + Exchange [ss_item_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 [c_customer_sk] + Sort [i_item_sk] InputAdapter - Exchange [c_customer_sk] #17 + Exchange [i_item_sk] #17 WholeStageCodegen (4) - 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 (9) - Sort [i_item_sk] + Sort [c_customer_sk] InputAdapter - Exchange [i_item_sk] #18 + Exchange [c_customer_sk] #18 WholeStageCodegen (8) - 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 (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 3100e574e60e3..a531797a20c7e 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 (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 (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 (51) +- Exchange (50) +- * Project (49) - +- * 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) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Filter (46) + : +- * ColumnarToRow (45) + : +- Scan parquet default.catalog_sales (44) + +- ReusedExchange (47) (1) Scan parquet default.store_sales @@ -224,76 +224,76 @@ 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, 5), ENSURE_REQUIREMENTS, [id=#30] +Arguments: hashpartitioning(sr_customer_sk#23, sr_item_sk#22, sr_ticket_number#24, 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), ENSURE_REQUIREMENTS, [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 -(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] +(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] 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 -(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)) +(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] -(43) ReusedExchange [Reuses operator id: 35] -Output [1]: [d_date_sk#35] +(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)) -(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] +(47) ReusedExchange [Reuses operator id: 35] +Output [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)] +(48) BroadcastHashJoin [codegen id : 15] +Left keys [1]: [cs_sold_date_sk#32] +Right keys [1]: [d_date_sk#36] Join condition: None (49) Project [codegen id : 15] -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] +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] (50) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#37] (51) Sort [codegen id : 16] -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 +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 (52) SortMergeJoin [codegen id : 17] -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] +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)] Join condition: None (53) Project [codegen id : 17] -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] +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] (54) HashAggregate [codegen id : 17] -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] +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] 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#34))] +Functions [3]: [partial_sum(UnscaledValue(ss_net_profit#6)), partial_sum(UnscaledValue(sr_net_loss#25)), partial_sum(UnscaledValue(cs_net_profit#35))] 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] @@ -304,9 +304,9 @@ Arguments: hashpartitioning(i_item_id#17, i_item_desc#18, s_store_id#12, s_store (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#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] +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] (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 9b53cdaa5dc67..ad9fa718ff2bd 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 [ss_customer_sk,ss_item_sk,ss_ticket_number,sr_customer_sk,sr_item_sk,sr_ticket_number] + SortMergeJoin [sr_customer_sk,sr_item_sk,cs_bill_customer_sk,cs_item_sk] InputAdapter - WholeStageCodegen (8) - Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] + WholeStageCodegen (13) + Sort [sr_customer_sk,sr_item_sk] InputAdapter - 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] + 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] InputAdapter - WholeStageCodegen (4) - Sort [ss_item_sk] + WholeStageCodegen (8) + Sort [ss_customer_sk,ss_item_sk,ss_ticket_number] InputAdapter - 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.date_dim [d_date_sk,d_year,d_moy] + 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 - 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 + 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 - 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] + WholeStageCodegen (6) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #7 + WholeStageCodegen (5) + Filter [i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc] InputAdapter WholeStageCodegen (11) - Sort [sr_customer_sk,sr_item_sk] + Sort [sr_customer_sk,sr_item_sk,sr_ticket_number] InputAdapter - Exchange [sr_customer_sk,sr_item_sk] #8 + Exchange [sr_customer_sk,sr_item_sk,sr_ticket_number] #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 - 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 + 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 cb8522545f1d3..2aa99626920ec 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 (17) - : : +- * BroadcastHashJoin Inner BuildRight (16) + : :- * Project (22) + : : +- * BroadcastHashJoin Inner BuildRight (21) : : :- * 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 (15) - : : +- * Project (14) - : : +- * Filter (13) - : : +- * ColumnarToRow (12) - : : +- Scan parquet default.customer_address (11) + : : +- 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 (27) - : +- * 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) + : +- * Project (26) + : +- * Filter (25) + : +- * ColumnarToRow (24) + : +- Scan parquet default.customer_address (23) :- * HashAggregate (47) : +- Exchange (46) : +- * HashAggregate (45) @@ -69,310 +69,327 @@ TakeOrderedAndProject (67) (1) Scan parquet default.store_sales -Output [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Output [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] Batched: true -Location [not included in comparison]/{warehouse_dir}/store_sales] -PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ss_sold_date_sk#4), dynamicpruningexpression(ss_sold_date_sk#4 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ss_addr_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] (3) Filter [codegen id : 5] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_addr_sk#3, ss_ext_sales_price#4] -Condition : ((isnotnull(ss_sold_date_sk#1) AND isnotnull(ss_addr_sk#3)) AND isnotnull(ss_item_sk#2)) +Input [4]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4] +Condition : (isnotnull(ss_addr_sk#2) AND isnotnull(ss_item_sk#1)) (4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [3]: [d_date_sk#6, d_year#7, d_moy#8] 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,5), IsNotNull(d_date_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (6) Filter [codegen id : 1] -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 = 5)) AND isnotnull(d_date_sk#5)) +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] +Condition : ((((isnotnull(d_year#7) AND isnotnull(d_moy#8)) AND (d_year#7 = 1998)) AND (d_moy#8 = 5)) AND isnotnull(d_date_sk#6)) (7) Project [codegen id : 1] -Output [1]: [d_date_sk#5] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Output [1]: [d_date_sk#6] +Input [3]: [d_date_sk#6, d_year#7, d_moy#8] (8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#8] +Input [1]: [d_date_sk#6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#9] (9) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ss_sold_date_sk#4] +Right keys [1]: [d_date_sk#6] Join condition: None (10) Project [codegen id : 5] -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] +Output [3]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3] +Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, ss_sold_date_sk#4, d_date_sk#6] -(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] +(11) Scan parquet default.item +Output [2]: [i_item_sk#10, i_manufact_id#11] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(19) ColumnarToRow [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] +(12) ColumnarToRow [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] -(20) Filter [codegen id : 4] -Input [2]: [i_item_sk#12, i_manufact_id#13] -Condition : isnotnull(i_item_sk#12) +(13) Filter [codegen id : 3] +Input [2]: [i_item_sk#10, i_manufact_id#11] +Condition : isnotnull(i_item_sk#10) -(21) Scan parquet default.item -Output [2]: [i_category#14, i_manufact_id#13] +(14) Scan parquet default.item +Output [2]: [i_category#12, i_manufact_id#13] Batched: true Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics)] +PushedFilters: [IsNotNull(i_category), EqualTo(i_category,Electronics )] ReadSchema: struct -(22) ColumnarToRow [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] +(15) ColumnarToRow [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] -(23) Filter [codegen id : 3] -Input [2]: [i_category#14, i_manufact_id#13] -Condition : (isnotnull(i_category#14) AND (i_category#14 = Electronics)) +(16) Filter [codegen id : 2] +Input [2]: [i_category#12, i_manufact_id#13] +Condition : (isnotnull(i_category#12) AND (i_category#12 = Electronics )) -(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] +(17) Project [codegen id : 2] +Output [1]: [i_manufact_id#13] +Input [2]: [i_category#12, i_manufact_id#13] -(25) BroadcastExchange -Input [1]: [i_manufact_id#13#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +(18) BroadcastExchange +Input [1]: [i_manufact_id#13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(26) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [i_manufact_id#13] -Right keys [1]: [i_manufact_id#13#15] +(19) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [i_manufact_id#11] +Right keys [1]: [i_manufact_id#13] Join condition: None +(20) BroadcastExchange +Input [2]: [i_item_sk#10, i_manufact_id#11] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#15] + +(21) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [ss_item_sk#1] +Right keys [1]: [i_item_sk#10] +Join condition: None + +(22) Project [codegen id : 5] +Output [3]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11] +Input [5]: [ss_item_sk#1, ss_addr_sk#2, ss_ext_sales_price#3, i_item_sk#10, i_manufact_id#11] + +(23) Scan parquet default.customer_address +Output [2]: [ca_address_sk#16, ca_gmt_offset#17] +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#16, ca_gmt_offset#17] + +(25) Filter [codegen id : 4] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] +Condition : ((isnotnull(ca_gmt_offset#17) AND (ca_gmt_offset#17 = -5.00)) AND isnotnull(ca_address_sk#16)) + +(26) Project [codegen id : 4] +Output [1]: [ca_address_sk#16] +Input [2]: [ca_address_sk#16, ca_gmt_offset#17] + (27) BroadcastExchange -Input [2]: [i_item_sk#12, i_manufact_id#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#17] +Input [1]: [ca_address_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (28) BroadcastHashJoin [codegen id : 5] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ss_addr_sk#2] +Right keys [1]: [ca_address_sk#16] Join condition: None (29) Project [codegen id : 5] -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] +Output [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Input [4]: [ss_addr_sk#2, ss_ext_sales_price#3, i_manufact_id#11, ca_address_sk#16] (30) HashAggregate [codegen id : 5] -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#13, sum#19] +Input [2]: [ss_ext_sales_price#3, i_manufact_id#11] +Keys [1]: [i_manufact_id#11] +Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum#19] +Results [2]: [i_manufact_id#11, sum#20] (31) Exchange -Input [2]: [i_manufact_id#13, sum#19] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#20] +Input [2]: [i_manufact_id#11, sum#20] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#21] (32) HashAggregate [codegen id : 6] -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#13, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#4))#21,17,2) AS total_sales#22] +Input [2]: [i_manufact_id#11, sum#20] +Keys [1]: [i_manufact_id#11] +Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#22] +Results [2]: [i_manufact_id#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#22,17,2) AS total_sales#23] (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] +Output [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_sold_date_sk), IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#27), dynamicpruningexpression(cs_sold_date_sk#27 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(cs_bill_addr_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (34) ColumnarToRow [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, 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, cs_sold_date_sk#27] (35) Filter [codegen id : 11] -Input [4]: [cs_sold_date_sk#23, cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26] -Condition : ((isnotnull(cs_sold_date_sk#23) AND isnotnull(cs_bill_addr_sk#24)) AND isnotnull(cs_item_sk#25)) +Input [4]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27] +Condition : (isnotnull(cs_bill_addr_sk#24) AND isnotnull(cs_item_sk#25)) (36) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#28] (37) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_sold_date_sk#23] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [cs_sold_date_sk#27] +Right keys [1]: [d_date_sk#28] Join condition: None (38) Project [codegen id : 11] 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] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, cs_sold_date_sk#27, d_date_sk#28] -(39) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] +(39) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#29, i_manufact_id#30] (40) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_bill_addr_sk#24] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [cs_item_sk#25] +Right keys [1]: [i_item_sk#29] Join condition: None (41) Project [codegen id : 11] -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] +Output [3]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30] +Input [5]: [cs_bill_addr_sk#24, cs_item_sk#25, cs_ext_sales_price#26, i_item_sk#29, i_manufact_id#30] (42) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] +Output [1]: [ca_address_sk#31] (43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [cs_item_sk#25] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [cs_bill_addr_sk#24] +Right keys [1]: [ca_address_sk#31] Join condition: None (44) Project [codegen id : 11] -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] +Output [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Input [4]: [cs_bill_addr_sk#24, cs_ext_sales_price#26, i_manufact_id#30, ca_address_sk#31] (45) HashAggregate [codegen id : 11] -Input [2]: [cs_ext_sales_price#26, i_manufact_id#13] -Keys [1]: [i_manufact_id#13] +Input [2]: [cs_ext_sales_price#26, i_manufact_id#30] +Keys [1]: [i_manufact_id#30] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#26))] -Aggregate Attributes [1]: [sum#27] -Results [2]: [i_manufact_id#13, sum#28] +Aggregate Attributes [1]: [sum#32] +Results [2]: [i_manufact_id#30, sum#33] (46) Exchange -Input [2]: [i_manufact_id#13, sum#28] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#29] +Input [2]: [i_manufact_id#30, sum#33] +Arguments: hashpartitioning(i_manufact_id#30, 5), ENSURE_REQUIREMENTS, [id=#34] (47) HashAggregate [codegen id : 12] -Input [2]: [i_manufact_id#13, sum#28] -Keys [1]: [i_manufact_id#13] +Input [2]: [i_manufact_id#30, sum#33] +Keys [1]: [i_manufact_id#30] 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#13, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#30,17,2) AS total_sales#31] +Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#26))#35] +Results [2]: [i_manufact_id#30, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#26))#35,17,2) AS total_sales#36] (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] +Output [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] Batched: true -Location [not included in comparison]/{warehouse_dir}/web_sales] -PushedFilters: [IsNotNull(ws_sold_date_sk), IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(ws_sold_date_sk#40), dynamicpruningexpression(ws_sold_date_sk#40 IN dynamicpruning#5)] +PushedFilters: [IsNotNull(ws_bill_addr_sk), IsNotNull(ws_item_sk)] +ReadSchema: struct (49) ColumnarToRow [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] (50) Filter [codegen id : 17] -Input [4]: [ws_sold_date_sk#32, ws_item_sk#33, ws_bill_addr_sk#34, ws_ext_sales_price#35] -Condition : ((isnotnull(ws_sold_date_sk#32) AND isnotnull(ws_bill_addr_sk#34)) AND isnotnull(ws_item_sk#33)) +Input [4]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40] +Condition : (isnotnull(ws_bill_addr_sk#38) AND isnotnull(ws_item_sk#37)) (51) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +Output [1]: [d_date_sk#41] (52) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_sold_date_sk#32] -Right keys [1]: [d_date_sk#5] +Left keys [1]: [ws_sold_date_sk#40] +Right keys [1]: [d_date_sk#41] Join condition: None (53) Project [codegen id : 17] -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] +Output [3]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, ws_sold_date_sk#40, d_date_sk#41] -(54) ReusedExchange [Reuses operator id: 15] -Output [1]: [ca_address_sk#9] +(54) ReusedExchange [Reuses operator id: 20] +Output [2]: [i_item_sk#42, i_manufact_id#43] (55) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_bill_addr_sk#34] -Right keys [1]: [ca_address_sk#9] +Left keys [1]: [ws_item_sk#37] +Right keys [1]: [i_item_sk#42] Join condition: None (56) Project [codegen id : 17] -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] +Output [3]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43] +Input [5]: [ws_item_sk#37, ws_bill_addr_sk#38, ws_ext_sales_price#39, i_item_sk#42, i_manufact_id#43] (57) ReusedExchange [Reuses operator id: 27] -Output [2]: [i_item_sk#12, i_manufact_id#13] +Output [1]: [ca_address_sk#44] (58) BroadcastHashJoin [codegen id : 17] -Left keys [1]: [ws_item_sk#33] -Right keys [1]: [i_item_sk#12] +Left keys [1]: [ws_bill_addr_sk#38] +Right keys [1]: [ca_address_sk#44] Join condition: None (59) Project [codegen id : 17] -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] +Output [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Input [4]: [ws_bill_addr_sk#38, ws_ext_sales_price#39, i_manufact_id#43, ca_address_sk#44] (60) HashAggregate [codegen id : 17] -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#13, sum#37] +Input [2]: [ws_ext_sales_price#39, i_manufact_id#43] +Keys [1]: [i_manufact_id#43] +Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum#45] +Results [2]: [i_manufact_id#43, sum#46] (61) Exchange -Input [2]: [i_manufact_id#13, sum#37] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#38] +Input [2]: [i_manufact_id#43, sum#46] +Arguments: hashpartitioning(i_manufact_id#43, 5), ENSURE_REQUIREMENTS, [id=#47] (62) HashAggregate [codegen id : 18] -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#13, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#35))#39,17,2) AS total_sales#40] +Input [2]: [i_manufact_id#43, sum#46] +Keys [1]: [i_manufact_id#43] +Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#39))] +Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#39))#48] +Results [2]: [i_manufact_id#43, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#39))#48,17,2) AS total_sales#49] (63) Union (64) HashAggregate [codegen id : 19] -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#13, sum#43, isEmpty#44] +Input [2]: [i_manufact_id#11, total_sales#23] +Keys [1]: [i_manufact_id#11] +Functions [1]: [partial_sum(total_sales#23)] +Aggregate Attributes [2]: [sum#50, isEmpty#51] +Results [3]: [i_manufact_id#11, sum#52, isEmpty#53] (65) Exchange -Input [3]: [i_manufact_id#13, sum#43, isEmpty#44] -Arguments: hashpartitioning(i_manufact_id#13, 5), ENSURE_REQUIREMENTS, [id=#45] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Arguments: hashpartitioning(i_manufact_id#11, 5), ENSURE_REQUIREMENTS, [id=#54] (66) HashAggregate [codegen id : 20] -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#13, sum(total_sales#22)#46 AS total_sales#47] +Input [3]: [i_manufact_id#11, sum#52, isEmpty#53] +Keys [1]: [i_manufact_id#11] +Functions [1]: [sum(total_sales#23)] +Aggregate Attributes [1]: [sum(total_sales#23)#55] +Results [2]: [i_manufact_id#11, sum(total_sales#23)#55 AS total_sales#56] (67) TakeOrderedAndProject -Input [2]: [i_manufact_id#13, total_sales#47] -Arguments: 100, [total_sales#47 ASC NULLS FIRST], [i_manufact_id#13, total_sales#47] +Input [2]: [i_manufact_id#11, total_sales#56] +Arguments: 100, [total_sales#56 ASC NULLS FIRST], [i_manufact_id#11, total_sales#56] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 +ReusedExchange (68) + + +(68) ReusedExchange [Reuses operator id: 8] +Output [1]: [d_date_sk#6] + +Subquery:2 Hosting operator id = 33 Hosting Expression = cs_sold_date_sk#27 IN dynamicpruning#5 + +Subquery:3 Hosting operator id = 48 Hosting Expression = ws_sold_date_sk#40 IN dynamicpruning#5 + 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 14787f0bbce7b..410def2466e1a 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_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_addr_sk,ca_address_sk] + 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] 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 (2) - Project [ca_address_sk] - Filter [ca_gmt_offset,ca_address_sk] + WholeStageCodegen (3) + 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 #5 + WholeStageCodegen (2) + Project [i_manufact_id] + Filter [i_category] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_category,i_manufact_id] InputAdapter - BroadcastExchange #5 + BroadcastExchange #6 WholeStageCodegen (4) - BroadcastHashJoin [i_manufact_id,i_manufact_id] - Filter [i_item_sk] + 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 #6 - WholeStageCodegen (3) - 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] 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_item_sk,i_item_sk] - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_bill_addr_sk,ca_address_sk] + 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] 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 [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 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_item_sk,i_item_sk] - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_bill_addr_sk,ca_address_sk] + 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] 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 [ca_address_sk] #4 + ReusedExchange [i_item_sk,i_manufact_id] #4 InputAdapter - ReusedExchange [i_item_sk,i_manufact_id] #5 + ReusedExchange [ca_address_sk] #6 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 6492918d3aa13..036de304b72cc 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 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) + : +- * 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.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] 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}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(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)) +(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)) -(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] +(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 -(5) BroadcastExchange -Input [2]: [d_date_sk#1, d_year#2] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(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 +(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)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(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] -(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)) +(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] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -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] +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] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] 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 +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 (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -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)) +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)) (14) Project [codegen id : 2] -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] +Output [2]: [d_date_sk#9, d_year#10] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [2]: [d_date_sk#9, d_year#10] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -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] +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] (18) HashAggregate [codegen id : 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))] +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))] Aggregate Attributes [1]: [sum#13] -Results [4]: [d_year#2, i_brand#10, i_brand_id#9, sum#14] +Results [4]: [d_year#10, i_brand#6, i_brand_id#5, sum#14] (19) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -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] +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] (21) TakeOrderedAndProject -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] +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] 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 f4aaf3df75135..8ed500d84390c 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_item_sk,i_item_sk] - Project [d_year,ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + 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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - 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] - 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - 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] 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 b8d8aa358d532..cca43a4232a8a 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 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) + : +- * 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.item (11) + +- Scan parquet default.date_dim (11) -(1) Scan parquet default.date_dim -Output [3]: [d_date_sk#1, d_year#2, d_moy#3] +(1) Scan parquet default.store_sales +Output [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] 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}/store_sales] +PushedFilters: [IsNotNull(ss_sold_date_sk), IsNotNull(ss_item_sk)] +ReadSchema: struct -(2) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(2) ColumnarToRow [codegen id : 3] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(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)) +(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)) -(4) Project [codegen id : 1] -Output [1]: [d_date_sk#1] -Input [3]: [d_date_sk#1, d_year#2, d_moy#3] +(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 -(5) BroadcastExchange -Input [1]: [d_date_sk#1] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#4] +(5) ColumnarToRow [codegen id : 1] +Input [4]: [i_item_sk#4, i_brand_id#5, i_brand#6, i_manager_id#7] -(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 +(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)) -(7) ColumnarToRow -Input [3]: [ss_sold_date_sk#5, ss_item_sk#6, ss_ext_sales_price#7] +(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] -(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)) +(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] (9) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [d_date_sk#1] -Right keys [1]: [ss_sold_date_sk#5] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#4] Join condition: None (10) Project [codegen id : 3] -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] +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] -(11) Scan parquet default.item -Output [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +(11) Scan parquet default.date_dim +Output [3]: [d_date_sk#9, d_year#10, d_moy#11] 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 +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 (12) ColumnarToRow [codegen id : 2] -Input [4]: [i_item_sk#8, i_brand_id#9, i_brand#10, i_manager_id#11] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (13) Filter [codegen id : 2] -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)) +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)) (14) Project [codegen id : 2] -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] +Output [1]: [d_date_sk#9] +Input [3]: [d_date_sk#9, d_year#10, d_moy#11] (15) BroadcastExchange -Input [3]: [i_item_sk#8, i_brand_id#9, i_brand#10] +Input [1]: [d_date_sk#9] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#12] (16) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [ss_item_sk#6] -Right keys [1]: [i_item_sk#8] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#9] Join condition: None (17) Project [codegen id : 3] -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] +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] (18) HashAggregate [codegen id : 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))] +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))] Aggregate Attributes [1]: [sum#13] -Results [3]: [i_brand#10, i_brand_id#9, sum#14] +Results [3]: [i_brand#6, i_brand_id#5, sum#14] (19) Exchange -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] +Input [3]: [i_brand#6, i_brand_id#5, sum#14] +Arguments: hashpartitioning(i_brand#6, i_brand_id#5, 5), ENSURE_REQUIREMENTS, [id=#15] (20) HashAggregate [codegen id : 4] -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] +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] (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 4f375c80678e8..b0d0e0d809441 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_item_sk,i_item_sk] - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [d_date_sk,ss_sold_date_sk] + 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] InputAdapter BroadcastExchange #2 WholeStageCodegen (1) - 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] - 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] + Scan parquet default.item [i_item_sk,i_brand_id,i_brand,i_manager_id] InputAdapter BroadcastExchange #3 WholeStageCodegen (2) - 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] 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 3007b11a1a860..3a61d77f674f0 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 (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 (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 (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * 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) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot 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), EqualTo(hd_buy_potential,>10000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,>10000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = >10000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] 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) 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] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(36) ColumnarToRow [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(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)) +(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)) -(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] +(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] -(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] +(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] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(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] +(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] -(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] +(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), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(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 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 -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] 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 [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] +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] (57) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -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 +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 (59) SortMergeJoin [codegen id : 16] -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)) +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) (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#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] +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] (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#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] +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] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +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), 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#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -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#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] +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] (76) HashAggregate [codegen id : 20] -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] +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] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -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] +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] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -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] +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] (79) TakeOrderedAndProject -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] +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] 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 b88505ad7b9bc..918508787c4b0 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,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - 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 + 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] 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] + 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] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + 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] 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 #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] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - 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] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow InputAdapter - 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] + 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 6813696266ac5..f89e4b8d093f1 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) - +- * 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] + +- * 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] 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 +Location [not included in comparison]/{warehouse_dir}/customer] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 2] -Input [4]: [cr_returned_date_sk#1, cr_returning_customer_sk#2, cr_returning_addr_sk#3, cr_return_amt_inc_tax#4] +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] (3) Filter [codegen id : 2] -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)) +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)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#5, d_year#6] +(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] 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 +Location [not included in comparison]/{warehouse_dir}/customer_address] +PushedFilters: [IsNotNull(ca_state), EqualTo(ca_state,GA), IsNotNull(ca_address_sk)] +ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#5, d_year#6] +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] (6) Filter [codegen id : 1] -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] +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)) -(8) BroadcastExchange -Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#7] +(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] -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] +(8) BroadcastHashJoin [codegen id : 2] +Left keys [1]: [c_current_addr_sk#3] +Right keys [1]: [ca_address_sk#7] Join condition: None -(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] +(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] -(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] +(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), ENSURE_REQUIREMENTS, [id=#20] -(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 +(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 -(13) Scan parquet default.customer_address -Output [2]: [ca_address_sk#9, ca_state#10] +(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] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_address] -PushedFilters: [IsNotNull(ca_address_sk), IsNotNull(ca_state)] -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 -(14) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#9, ca_state#10] +(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] -(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)) +(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)) -(16) Exchange -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: hashpartitioning(ca_address_sk#9, 5), ENSURE_REQUIREMENTS, [id=#11] +(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 -(17) Sort [codegen id : 5] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(16) ColumnarToRow [codegen id : 4] +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 +(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)) -(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 +(18) Project [codegen id : 4] +Output [1]: [d_date_sk#25] +Input [2]: [d_date_sk#25, d_year#26] -(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] +(19) BroadcastExchange +Input [1]: [d_date_sk#25] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#27] -(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)) +(20) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] +Join condition: None -(27) ReusedExchange [Reuses operator id: 8] -Output [1]: [d_date_sk#5] +(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] -(28) BroadcastHashJoin [codegen id : 8] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] -Join condition: None +(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), ENSURE_REQUIREMENTS, [id=#28] -(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] +(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 -(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] +(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 -(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 +(25) ColumnarToRow [codegen id : 7] +Input [2]: [ca_address_sk#7, ca_state#14] -(32) ReusedExchange [Reuses operator id: 16] -Output [2]: [ca_address_sk#9, ca_state#10] +(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)) -(33) Sort [codegen id : 11] -Input [2]: [ca_address_sk#9, ca_state#10] -Arguments: [ca_address_sk#9 ASC NULLS FIRST], false, 0 +(27) Exchange +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: hashpartitioning(ca_address_sk#7, 5), ENSURE_REQUIREMENTS, [id=#29] -(34) SortMergeJoin [codegen id : 12] -Left keys [1]: [cr_returning_addr_sk#3] -Right keys [1]: [ca_address_sk#9] -Join condition: None +(28) Sort [codegen id : 8] +Input [2]: [ca_address_sk#7, ca_state#14] +Arguments: [ca_address_sk#7 ASC NULLS FIRST], false, 0 -(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 +(29) SortMergeJoin [codegen id : 9] +Left keys [1]: [cr_returning_addr_sk#23] +Right keys [1]: [ca_address_sk#7] +Join condition: None -(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] +(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), ENSURE_REQUIREMENTS, [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), ENSURE_REQUIREMENTS, [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 -(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)) +(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] -(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] +(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] 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}/catalog_returns] +PushedFilters: [IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_addr_sk)] +ReadSchema: struct -(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] +(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] -(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)) +(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)) -(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] +(42) ReusedExchange [Reuses operator id: 19] +Output [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] +(43) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [cr_returned_date_sk#21] +Right keys [1]: [d_date_sk#25] Join condition: None -(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] +(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] -(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] +(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), ENSURE_REQUIREMENTS, [id=#38] -(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 +(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 -(59) SortMergeJoin [codegen id : 20] -Left keys [1]: [ctr_customer_sk#16] -Right keys [1]: [c_customer_sk#34] +(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] 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), ENSURE_REQUIREMENTS, [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), ENSURE_REQUIREMENTS, [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#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] +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] (61) TakeOrderedAndProject -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] +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] 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 99677b6e39736..c603ab5194286 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,29 +1,48 @@ 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] - 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)))] + 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) 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] #2 - WholeStageCodegen (6) + Exchange [cr_returning_customer_sk,ca_state] #4 + WholeStageCodegen (9) 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 (3) + WholeStageCodegen (6) Sort [cr_returning_addr_sk] InputAdapter - Exchange [cr_returning_addr_sk] #3 - WholeStageCodegen (2) + Exchange [cr_returning_addr_sk] #5 + WholeStageCodegen (5) 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] @@ -31,74 +50,55 @@ 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 #4 - WholeStageCodegen (1) + BroadcastExchange #6 + WholeStageCodegen (4) Project [d_date_sk] Filter [d_year,d_date_sk] ColumnarToRow InputAdapter Scan parquet default.date_dim [d_date_sk,d_year] InputAdapter - WholeStageCodegen (5) + WholeStageCodegen (8) Sort [ca_address_sk] InputAdapter - Exchange [ca_address_sk] #5 - WholeStageCodegen (4) + Exchange [ca_address_sk] #7 + WholeStageCodegen (7) 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 - 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 + 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] 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 #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] + 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 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 6bcbe470cec50..16589f89deb6e 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 (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) + : :- * 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) : +- BroadcastExchange (34) - : +- * Filter (33) - : +- * ColumnarToRow (32) - : +- Scan parquet default.customer_demographics (31) + : +- * 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) +- BroadcastExchange (40) +- * Filter (39) +- * ColumnarToRow (38) +- Scan parquet default.call_center (37) -(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] +(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] 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] +PushedFilters: [IsNotNull(c_customer_sk), IsNotNull(c_current_addr_sk), IsNotNull(c_current_cdemo_sk), IsNotNull(c_current_hdemo_sk)] +ReadSchema: struct -(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] +(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] -(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)) +(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)) -(4) Scan parquet default.date_dim -Output [3]: [d_date_sk#5, d_year#6, d_moy#7] +(4) Scan parquet default.customer_demographics +Output [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] 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 +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 (5) ColumnarToRow [codegen id : 1] -Input [3]: [d_date_sk#5, d_year#6, d_moy#7] +Input [3]: [cd_demo_sk#5, cd_marital_status#6, cd_education_status#7] (6) Filter [codegen id : 1] -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)) +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)) -(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] - -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cr_returned_date_sk#1] -Right keys [1]: [d_date_sk#5] +(8) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#5] Join condition: None -(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] - -(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] +(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] -(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] +(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}/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}/household_demographics] +PushedFilters: [IsNotNull(hd_buy_potential), StringStartsWith(hd_buy_potential,Unknown), IsNotNull(hd_demo_sk)] +ReadSchema: struct + +(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)) -(13) ColumnarToRow -Input [4]: [c_customer_sk#10, c_current_cdemo_sk#11, c_current_hdemo_sk#12, c_current_addr_sk#13] +(13) Project [codegen id : 2] +Output [1]: [hd_demo_sk#9] +Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -(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)) +(14) BroadcastExchange +Input [1]: [hd_demo_sk#9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#11] (15) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [cr_returning_customer_sk#2] -Right keys [1]: [c_customer_sk#10] +Left keys [1]: [c_current_hdemo_sk#3] +Right keys [1]: [hd_demo_sk#9] Join condition: None (16) Project [codegen id : 7] -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] +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] -(17) Scan parquet default.household_demographics -Output [2]: [hd_demo_sk#14, hd_buy_potential#15] +(17) Scan parquet default.customer_address +Output [2]: [ca_address_sk#12, ca_gmt_offset#13] 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 +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 (18) ColumnarToRow [codegen id : 3] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (19) Filter [codegen id : 3] -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)) +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)) (20) Project [codegen id : 3] -Output [1]: [hd_demo_sk#14] -Input [2]: [hd_demo_sk#14, hd_buy_potential#15] +Output [1]: [ca_address_sk#12] +Input [2]: [ca_address_sk#12, ca_gmt_offset#13] (21) BroadcastExchange -Input [1]: [hd_demo_sk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#16] +Input [1]: [ca_address_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] (22) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_hdemo_sk#12] -Right keys [1]: [hd_demo_sk#14] +Left keys [1]: [c_current_addr_sk#4] +Right keys [1]: [ca_address_sk#12] Join condition: None (23) Project [codegen id : 7] -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] +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] -(24) Scan parquet default.customer_address -Output [2]: [ca_address_sk#17, ca_gmt_offset#18] +(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] 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}/catalog_returns] +PushedFilters: [IsNotNull(cr_call_center_sk), IsNotNull(cr_returned_date_sk), IsNotNull(cr_returning_customer_sk)] +ReadSchema: struct -(25) ColumnarToRow [codegen id : 4] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(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] -(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)) +(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)) -(27) Project [codegen id : 4] -Output [1]: [ca_address_sk#17] -Input [2]: [ca_address_sk#17, ca_gmt_offset#18] +(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 -(28) BroadcastExchange -Input [1]: [ca_address_sk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#19] +(28) ColumnarToRow [codegen id : 4] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_addr_sk#13] -Right keys [1]: [ca_address_sk#17] -Join condition: None +(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)) -(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] +(30) Project [codegen id : 4] +Output [1]: [d_date_sk#19] +Input [3]: [d_date_sk#19, d_year#20, d_moy#21] -(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 +(31) BroadcastExchange +Input [1]: [d_date_sk#19] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#22] -(32) ColumnarToRow [codegen id : 5] -Input [3]: [cd_demo_sk#20, cd_marital_status#21, cd_education_status#22] +(32) BroadcastHashJoin [codegen id : 5] +Left keys [1]: [cr_returned_date_sk#15] +Right keys [1]: [d_date_sk#19] +Join condition: None -(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)) +(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] (34) BroadcastExchange -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] +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] (35) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [c_current_cdemo_sk#11] -Right keys [1]: [cd_demo_sk#20] +Left keys [1]: [c_customer_sk#1] +Right keys [1]: [cr_returning_customer_sk#16] Join condition: None (36) Project [codegen id : 7] -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] +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] (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,31 +228,31 @@ 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#3] +Left keys [1]: [cr_call_center_sk#17] 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#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] +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] (43) HashAggregate [codegen id : 7] -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))] +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))] Aggregate Attributes [1]: [sum#29] -Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#21, cd_education_status#22, sum#30] +Results [6]: [cc_call_center_id#25, cc_name#26, cc_manager#27, cd_marital_status#6, cd_education_status#7, sum#30] (44) Exchange -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] +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), 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#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] +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] (46) Exchange 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 6c8d629feed3e..87beb3b565cc1 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 [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] + 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] BroadcastHashJoin [c_current_addr_sk,ca_address_sk] - Project [cr_call_center_sk,cr_net_loss,c_current_cdemo_sk,c_current_addr_sk] + Project [c_customer_sk,c_current_addr_sk,cd_marital_status,cd_education_status] BroadcastHashJoin [c_current_hdemo_sk,hd_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] + 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] 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 #5 - WholeStageCodegen (3) + BroadcastExchange #4 + WholeStageCodegen (2) 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 #6 - WholeStageCodegen (4) + BroadcastExchange #5 + WholeStageCodegen (3) 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 #7 + BroadcastExchange #6 WholeStageCodegen (5) - Filter [cd_marital_status,cd_education_status,cd_demo_sk] - ColumnarToRow + 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 - Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status,cd_education_status] + 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] InputAdapter BroadcastExchange #8 WholeStageCodegen (6) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt index 30dabdd2d5523..c005ef0f9ff1a 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt @@ -10,24 +10,24 @@ +- 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.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -37,126 +37,126 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#9] +Results [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21, i_item_id#6] (23) Exchange -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#9] -Input [9]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, i_item_id#9, _we0#23] +Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24, i_item_id#6] +Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, i_item_id#6, _we0#23] (27) Exchange -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 (29) Project [codegen id : 10] -Output [6]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Input [7]: [i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24, i_item_id#9] +Output [6]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24, i_item_id#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt index a180ed2b060b9..c6ce597f3c1bf 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/simplified.txt @@ -18,34 +18,34 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #6 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt index c1bf12b7c2c5a..60d89b18f8fc9 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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.web_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.web_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.web_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] Condition : (isnotnull(ws_item_sk#2) AND isnotnull(ws_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: hashpartitioning(ws_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3] +Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ws_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Input [4]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: hashpartitioning(ws_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ws_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ws_item_sk#2, ws_ext_sales_price#3] -Arguments: [ws_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ws_sold_date_sk#1, ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ws_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ws_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ws_item_sk#2, ws_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ws_sold_date_sk#1, ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ws_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ws_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ws_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ws_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt index d8db515f84f68..cfb9973ef6983 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ws_ext_sales_price] [sum,sum] Project [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ws_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ws_item_sk] - InputAdapter - Exchange [ws_item_sk] #3 - WholeStageCodegen (2) - Project [ws_item_sk,ws_ext_sales_price] - BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + BroadcastHashJoin [ws_sold_date_sk,d_date_sk] + Project [ws_sold_date_sk,ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ws_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ws_item_sk] + InputAdapter + Exchange [ws_item_sk] #3 + WholeStageCodegen (1) Filter [ws_item_sk,ws_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.web_sales [ws_sold_date_sk,ws_item_sk,ws_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 f6c5258701525..35e3304de7082 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) - : +- * 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) + : +- * 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) :- * HashAggregate (76) : +- Exchange (75) : +- * HashAggregate (74) @@ -162,716 +162,733 @@ TakeOrderedAndProject (160) (1) 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] +Output [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#9), dynamicpruningexpression(cs_sold_date_sk#9 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (2) ColumnarToRow [codegen id : 4] -Input [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] +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] (3) Filter [codegen id : 4] -Input [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] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9] +Condition : ((isnotnull(cs_bill_cdemo_sk#2) AND isnotnull(cs_bill_customer_sk#1)) AND isnotnull(cs_item_sk#3)) (4) Scan parquet default.customer_demographics -Output [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College), IsNotNull(cd_demo_sk)] +PushedFilters: [IsNotNull(cd_gender), IsNotNull(cd_education_status), EqualTo(cd_gender,M), EqualTo(cd_education_status,College ), IsNotNull(cd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (6) Filter [codegen id : 1] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] -Condition : ((((isnotnull(cd_gender#11) AND isnotnull(cd_education_status#12)) AND (cd_gender#11 = M)) AND (cd_education_status#12 = College)) AND isnotnull(cd_demo_sk#10)) +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] +Condition : ((((isnotnull(cd_gender#12) AND isnotnull(cd_education_status#13)) AND (cd_gender#12 = M)) AND (cd_education_status#13 = College )) AND isnotnull(cd_demo_sk#11)) (7) Project [codegen id : 1] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] -Input [4]: [cd_demo_sk#10, cd_gender#11, cd_education_status#12, cd_dep_count#13] +Output [2]: [cd_demo_sk#11, cd_dep_count#14] +Input [4]: [cd_demo_sk#11, cd_gender#12, cd_education_status#13, cd_dep_count#14] (8) BroadcastExchange -Input [2]: [cd_demo_sk#10, cd_dep_count#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] +Input [2]: [cd_demo_sk#11, cd_dep_count#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#15] (9) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#2] +Right keys [1]: [cd_demo_sk#11] Join condition: None (10) Project [codegen id : 4] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [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, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14] +Input [11]: [cs_bill_customer_sk#1, cs_bill_cdemo_sk#2, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_demo_sk#11, cd_dep_count#14] (11) Scan parquet default.date_dim -Output [2]: [d_date_sk#15, d_year#16] +Output [2]: [d_date_sk#16, d_year#17] 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 (12) ColumnarToRow [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] +Input [2]: [d_date_sk#16, d_year#17] (13) Filter [codegen id : 2] -Input [2]: [d_date_sk#15, d_year#16] -Condition : ((isnotnull(d_year#16) AND (d_year#16 = 2001)) AND isnotnull(d_date_sk#15)) +Input [2]: [d_date_sk#16, d_year#17] +Condition : ((isnotnull(d_year#17) AND (d_year#17 = 2001)) AND isnotnull(d_date_sk#16)) (14) Project [codegen id : 2] -Output [1]: [d_date_sk#15] -Input [2]: [d_date_sk#15, d_year#16] +Output [1]: [d_date_sk#16] +Input [2]: [d_date_sk#16, d_year#17] (15) BroadcastExchange -Input [1]: [d_date_sk#15] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#17] +Input [1]: [d_date_sk#16] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#18] (16) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#9] +Right keys [1]: [d_date_sk#16] Join condition: None (17) Project [codegen id : 4] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cs_sold_date_sk#9, cd_dep_count#14, d_date_sk#16] (18) Scan parquet default.item -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#19, i_item_id#20] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (19) ColumnarToRow [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] +Input [2]: [i_item_sk#19, i_item_id#20] (20) Filter [codegen id : 3] -Input [2]: [i_item_sk#18, i_item_id#19] -Condition : isnotnull(i_item_sk#18) +Input [2]: [i_item_sk#19, i_item_id#20] +Condition : isnotnull(i_item_sk#19) (21) BroadcastExchange -Input [2]: [i_item_sk#18, i_item_id#19] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#20] +Input [2]: [i_item_sk#19, i_item_id#20] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#21] (22) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#3] +Right keys [1]: [i_item_sk#19] Join condition: None (23) Project [codegen id : 4] -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] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, 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_sk#18, i_item_id#19] +Output [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Input [10]: [cs_bill_customer_sk#1, cs_item_sk#3, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_sk#19, i_item_id#20] (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), ENSURE_REQUIREMENTS, [id=#21] +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: hashpartitioning(cs_bill_customer_sk#1, 5), ENSURE_REQUIREMENTS, [id=#22] (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] -Arguments: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20] +Arguments: [cs_bill_customer_sk#1 ASC NULLS FIRST], false, 0 (26) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] Batched: true 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 : 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] +(27) ColumnarToRow [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(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)) +(28) Filter [codegen id : 7] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] +Condition : (((c_birth_month#26 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#23)) AND isnotnull(c_current_cdemo_sk#24)) AND isnotnull(c_current_addr_sk#25)) -(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] +(29) Project [codegen id : 7] +Output [4]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27] +Input [5]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_month#26, c_birth_year#27] -(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] - -(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 - -(32) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#28] +(30) Scan parquet default.customer_address +Output [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] Batched: true -Location [not included in comparison]/{warehouse_dir}/customer_demographics] -PushedFilters: [IsNotNull(cd_demo_sk)] -ReadSchema: struct - -(33) ColumnarToRow [codegen id : 8] -Input [1]: [cd_demo_sk#28] +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 -(34) Filter [codegen id : 8] -Input [1]: [cd_demo_sk#28] -Condition : isnotnull(cd_demo_sk#28) +(31) ColumnarToRow [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] -(35) Exchange -Input [1]: [cd_demo_sk#28] -Arguments: hashpartitioning(cd_demo_sk#28, 5), ENSURE_REQUIREMENTS, [id=#29] +(32) Filter [codegen id : 6] +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Condition : (ca_state#30 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#28)) -(36) Sort [codegen id : 9] -Input [1]: [cd_demo_sk#28] -Arguments: [cd_demo_sk#28 ASC NULLS FIRST], false, 0 +(33) BroadcastExchange +Input [4]: [ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#32] -(37) SortMergeJoin [codegen id : 11] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#28] +(34) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [c_current_addr_sk#25] +Right keys [1]: [ca_address_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] +(35) Project [codegen id : 7] +Output [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [8]: [c_customer_sk#23, c_current_cdemo_sk#24, c_current_addr_sk#25, c_birth_year#27, ca_address_sk#28, ca_county#29, ca_state#30, ca_country#31] + +(36) Exchange +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_current_cdemo_sk#24, 5), ENSURE_REQUIREMENTS, [id=#33] -(39) Scan parquet default.customer_address -Output [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(37) Sort [codegen id : 8] +Input [6]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_current_cdemo_sk#24 ASC NULLS FIRST], false, 0 + +(38) Scan parquet default.customer_demographics +Output [1]: [cd_demo_sk#34] 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 +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#34] -(40) ColumnarToRow [codegen id : 10] -Input [4]: [ca_address_sk#30, ca_county#31, ca_state#32, ca_country#33] +(40) Filter [codegen id : 9] +Input [1]: [cd_demo_sk#34] +Condition : isnotnull(cd_demo_sk#34) -(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)) +(41) Exchange +Input [1]: [cd_demo_sk#34] +Arguments: hashpartitioning(cd_demo_sk#34, 5), ENSURE_REQUIREMENTS, [id=#35] -(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] +(42) Sort [codegen id : 10] +Input [1]: [cd_demo_sk#34] +Arguments: [cd_demo_sk#34 ASC NULLS FIRST], false, 0 -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +(43) SortMergeJoin [codegen id : 11] +Left keys [1]: [c_current_cdemo_sk#24] +Right keys [1]: [cd_demo_sk#34] Join condition: None (44) Project [codegen id : 11] -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] +Output [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Input [7]: [c_customer_sk#23, c_current_cdemo_sk#24, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31, cd_demo_sk#34] (45) Exchange -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] +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: hashpartitioning(c_customer_sk#23, 5), ENSURE_REQUIREMENTS, [id=#36] (46) Sort [codegen id : 12] -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 +Input [5]: [c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] +Arguments: [c_customer_sk#23 ASC NULLS FIRST], false, 0 (47) SortMergeJoin [codegen id : 13] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#1] +Right keys [1]: [c_customer_sk#23] Join condition: None (48) Project [codegen id : 13] -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] +Output [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, cast(cs_quantity#4 as decimal(12,2)) AS agg1#37, cast(cs_list_price#5 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#7 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#6 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#8 as decimal(12,2)) AS agg5#41, cast(c_birth_year#27 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#14 as decimal(12,2)) AS agg7#43] +Input [13]: [cs_bill_customer_sk#1, cs_quantity#4, cs_list_price#5, cs_sales_price#6, cs_coupon_amt#7, cs_net_profit#8, cd_dep_count#14, i_item_id#20, c_customer_sk#23, c_birth_year#27, ca_county#29, ca_state#30, ca_country#31] (49) HashAggregate [codegen id : 13] -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#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] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#44, count#45, sum#46, count#47, sum#48, count#49, sum#50, count#51, sum#52, count#53, sum#54, count#55, sum#56, count#57] +Results [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] (50) Exchange -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] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Arguments: hashpartitioning(i_item_id#20, ca_country#31, ca_state#30, ca_county#29, 5), ENSURE_REQUIREMENTS, [id=#72] (51) HashAggregate [codegen id : 14] -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#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] +Input [18]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, sum#58, count#59, sum#60, count#61, sum#62, count#63, sum#64, count#65, sum#66, count#67, sum#68, count#69, sum#70, count#71] +Keys [4]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#73, avg(agg2#38)#74, avg(agg3#39)#75, avg(agg4#40)#76, avg(agg5#41)#77, avg(agg6#42)#78, avg(agg7#43)#79] +Results [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, avg(agg1#37)#73 AS agg1#80, avg(agg2#38)#74 AS agg2#81, avg(agg3#39)#75 AS agg3#82, avg(agg4#40)#76 AS agg4#83, avg(agg5#41)#77 AS agg5#84, avg(agg6#42)#78 AS agg6#85, avg(agg7#43)#79 AS agg7#86] -(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] +(52) ReusedExchange [Reuses operator id: unknown] +Output [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] (53) Sort [codegen id : 19] -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: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94] +Arguments: [cs_bill_customer_sk#87 ASC NULLS FIRST], false, 0 (54) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] Batched: true 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 (55) ColumnarToRow [codegen id : 21] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (56) Filter [codegen id : 21] -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)) +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] +Condition : (((c_birth_month#98 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#95)) AND isnotnull(c_current_cdemo_sk#96)) AND isnotnull(c_current_addr_sk#97)) (57) Project [codegen id : 21] -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] +Output [4]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_month#98, c_birth_year#99] (58) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#100, ca_state#101, ca_country#102] 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#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] (60) Filter [codegen id : 20] -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)) +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Condition : (ca_state#101 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#100)) (61) BroadcastExchange -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] +Input [3]: [ca_address_sk#100, ca_state#101, ca_country#102] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#103] (62) BroadcastHashJoin [codegen id : 21] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#97] +Right keys [1]: [ca_address_sk#100] 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#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] +Output [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Input [7]: [c_customer_sk#95, c_current_cdemo_sk#96, c_current_addr_sk#97, c_birth_year#99, ca_address_sk#100, ca_state#101, ca_country#102] (64) Exchange -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] +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_current_cdemo_sk#96, 5), ENSURE_REQUIREMENTS, [id=#104] (65) Sort [codegen id : 22] -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 +Input [5]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_current_cdemo_sk#96 ASC NULLS FIRST], false, 0 -(66) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#88] +(66) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#105] (67) Sort [codegen id : 24] -Input [1]: [cd_demo_sk#88] -Arguments: [cd_demo_sk#88 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#105] +Arguments: [cd_demo_sk#105 ASC NULLS FIRST], false, 0 (68) SortMergeJoin [codegen id : 25] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#88] +Left keys [1]: [c_current_cdemo_sk#96] +Right keys [1]: [cd_demo_sk#105] Join condition: None (69) Project [codegen id : 25] -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] +Output [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Input [6]: [c_customer_sk#95, c_current_cdemo_sk#96, c_birth_year#99, ca_state#101, ca_country#102, cd_demo_sk#105] (70) Exchange -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] +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: hashpartitioning(c_customer_sk#95, 5), ENSURE_REQUIREMENTS, [id=#106] (71) Sort [codegen id : 26] -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 +Input [4]: [c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] +Arguments: [c_customer_sk#95 ASC NULLS FIRST], false, 0 (72) SortMergeJoin [codegen id : 27] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#87] +Right keys [1]: [c_customer_sk#95] Join condition: None (73) Project [codegen id : 27] -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] +Output [10]: [i_item_id#94, ca_country#102, ca_state#101, cast(cs_quantity#88 as decimal(12,2)) AS agg1#37, cast(cs_list_price#89 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#91 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#90 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#92 as decimal(12,2)) AS agg5#41, cast(c_birth_year#99 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#93 as decimal(12,2)) AS agg7#43] +Input [12]: [cs_bill_customer_sk#87, cs_quantity#88, cs_list_price#89, cs_sales_price#90, cs_coupon_amt#91, cs_net_profit#92, cd_dep_count#93, i_item_id#94, c_customer_sk#95, c_birth_year#99, ca_state#101, ca_country#102] (74) HashAggregate [codegen id : 27] -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#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] +Input [10]: [i_item_id#94, ca_country#102, ca_state#101, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#107, count#108, sum#109, count#110, sum#111, count#112, sum#113, count#114, sum#115, count#116, sum#117, count#118, sum#119, count#120] +Results [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] (75) Exchange -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] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Arguments: hashpartitioning(i_item_id#94, ca_country#102, ca_state#101, 5), ENSURE_REQUIREMENTS, [id=#135] (76) HashAggregate [codegen id : 28] -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#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] +Input [17]: [i_item_id#94, ca_country#102, ca_state#101, sum#121, count#122, sum#123, count#124, sum#125, count#126, sum#127, count#128, sum#129, count#130, sum#131, count#132, sum#133, count#134] +Keys [3]: [i_item_id#94, ca_country#102, ca_state#101] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#136, avg(agg2#38)#137, avg(agg3#39)#138, avg(agg4#40)#139, avg(agg5#41)#140, avg(agg6#42)#141, avg(agg7#43)#142] +Results [11]: [i_item_id#94, ca_country#102, ca_state#101, null AS county#143, avg(agg1#37)#136 AS agg1#144, avg(agg2#38)#137 AS agg2#145, avg(agg3#39)#138 AS agg3#146, avg(agg4#40)#139 AS agg4#147, avg(agg5#41)#140 AS agg5#148, avg(agg6#42)#141 AS agg6#149, avg(agg7#43)#142 AS agg7#150] -(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] +(77) ReusedExchange [Reuses operator id: unknown] +Output [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] (78) Sort [codegen id : 33] -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: [cs_bill_customer_sk#2 ASC NULLS FIRST], false, 0 +Input [8]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158] +Arguments: [cs_bill_customer_sk#151 ASC NULLS FIRST], false, 0 (79) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] Batched: true 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 (80) ColumnarToRow [codegen id : 35] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (81) Filter [codegen id : 35] -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)) +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] +Condition : (((c_birth_month#162 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#159)) AND isnotnull(c_current_cdemo_sk#160)) AND isnotnull(c_current_addr_sk#161)) (82) Project [codegen id : 35] -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] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_month#162, c_birth_year#163] (83) Scan parquet default.customer_address -Output [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [3]: [ca_address_sk#164, ca_state#165, ca_country#166] 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#30, ca_state#32, ca_country#33] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (85) Filter [codegen id : 34] -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)) +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] +Condition : (ca_state#165 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#164)) (86) Project [codegen id : 34] -Output [2]: [ca_address_sk#30, ca_country#33] -Input [3]: [ca_address_sk#30, ca_state#32, ca_country#33] +Output [2]: [ca_address_sk#164, ca_country#166] +Input [3]: [ca_address_sk#164, ca_state#165, ca_country#166] (87) BroadcastExchange -Input [2]: [ca_address_sk#30, ca_country#33] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#134] +Input [2]: [ca_address_sk#164, ca_country#166] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#167] (88) BroadcastHashJoin [codegen id : 35] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#161] +Right keys [1]: [ca_address_sk#164] 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#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] +Output [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Input [6]: [c_customer_sk#159, c_current_cdemo_sk#160, c_current_addr_sk#161, c_birth_year#163, ca_address_sk#164, ca_country#166] (90) Exchange -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] +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_current_cdemo_sk#160, 5), ENSURE_REQUIREMENTS, [id=#168] (91) Sort [codegen id : 36] -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 +Input [4]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166] +Arguments: [c_current_cdemo_sk#160 ASC NULLS FIRST], false, 0 -(92) ReusedExchange [Reuses operator id: 35] -Output [1]: [cd_demo_sk#136] +(92) ReusedExchange [Reuses operator id: 41] +Output [1]: [cd_demo_sk#169] (93) Sort [codegen id : 38] -Input [1]: [cd_demo_sk#136] -Arguments: [cd_demo_sk#136 ASC NULLS FIRST], false, 0 +Input [1]: [cd_demo_sk#169] +Arguments: [cd_demo_sk#169 ASC NULLS FIRST], false, 0 (94) SortMergeJoin [codegen id : 39] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#136] +Left keys [1]: [c_current_cdemo_sk#160] +Right keys [1]: [cd_demo_sk#169] Join condition: None (95) Project [codegen id : 39] -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] +Output [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Input [5]: [c_customer_sk#159, c_current_cdemo_sk#160, c_birth_year#163, ca_country#166, cd_demo_sk#169] (96) Exchange -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: hashpartitioning(c_customer_sk#22, 5), ENSURE_REQUIREMENTS, [id=#137] +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: hashpartitioning(c_customer_sk#159, 5), ENSURE_REQUIREMENTS, [id=#170] (97) Sort [codegen id : 40] -Input [3]: [c_customer_sk#22, c_birth_year#26, ca_country#33] -Arguments: [c_customer_sk#22 ASC NULLS FIRST], false, 0 +Input [3]: [c_customer_sk#159, c_birth_year#163, ca_country#166] +Arguments: [c_customer_sk#159 ASC NULLS FIRST], false, 0 (98) SortMergeJoin [codegen id : 41] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#151] +Right keys [1]: [c_customer_sk#159] Join condition: None (99) Project [codegen id : 41] -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] +Output [9]: [i_item_id#158, ca_country#166, cast(cs_quantity#152 as decimal(12,2)) AS agg1#37, cast(cs_list_price#153 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#155 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#154 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#156 as decimal(12,2)) AS agg5#41, cast(c_birth_year#163 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#157 as decimal(12,2)) AS agg7#43] +Input [11]: [cs_bill_customer_sk#151, cs_quantity#152, cs_list_price#153, cs_sales_price#154, cs_coupon_amt#155, cs_net_profit#156, cd_dep_count#157, i_item_id#158, c_customer_sk#159, c_birth_year#163, ca_country#166] (100) HashAggregate [codegen id : 41] -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#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] +Input [9]: [i_item_id#158, ca_country#166, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [2]: [i_item_id#158, ca_country#166] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#171, count#172, sum#173, count#174, sum#175, count#176, sum#177, count#178, sum#179, count#180, sum#181, count#182, sum#183, count#184] +Results [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] (101) Exchange -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] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Arguments: hashpartitioning(i_item_id#158, ca_country#166, 5), ENSURE_REQUIREMENTS, [id=#199] (102) HashAggregate [codegen id : 42] -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#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] +Input [16]: [i_item_id#158, ca_country#166, sum#185, count#186, sum#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198] +Keys [2]: [i_item_id#158, ca_country#166] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#200, avg(agg2#38)#201, avg(agg3#39)#202, avg(agg4#40)#203, avg(agg5#41)#204, avg(agg6#42)#205, avg(agg7#43)#206] +Results [11]: [i_item_id#158, ca_country#166, null AS ca_state#207, null AS county#208, avg(agg1#37)#200 AS agg1#209, avg(agg2#38)#201 AS agg2#210, avg(agg3#39)#202 AS agg3#211, avg(agg4#40)#203 AS agg4#212, avg(agg5#41)#204 AS agg5#213, avg(agg6#42)#205 AS agg6#214, avg(agg7#43)#206 AS agg7#215] (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] +Output [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#224), dynamicpruningexpression(cs_sold_date_sk#224 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (104) ColumnarToRow [codegen id : 49] -Input [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] +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] (105) Filter [codegen id : 49] -Input [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] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224] +Condition : ((isnotnull(cs_bill_cdemo_sk#217) AND isnotnull(cs_bill_customer_sk#216)) AND isnotnull(cs_item_sk#218)) (106) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#225, cd_dep_count#226] (107) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#217] +Right keys [1]: [cd_demo_sk#225] Join condition: None (108) Project [codegen id : 49] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [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, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226] +Input [11]: [cs_bill_customer_sk#216, cs_bill_cdemo_sk#217, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_demo_sk#225, cd_dep_count#226] (109) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#227] (110) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#224] +Right keys [1]: [d_date_sk#227] Join condition: None (111) Project [codegen id : 49] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cs_sold_date_sk#224, cd_dep_count#226, d_date_sk#227] (112) Scan parquet default.customer -Output [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Output [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] Batched: true 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 (113) ColumnarToRow [codegen id : 46] -Input [5]: [c_customer_sk#22, c_current_cdemo_sk#23, c_current_addr_sk#24, c_birth_month#25, c_birth_year#26] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (114) Filter [codegen id : 46] -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)) +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] +Condition : (((c_birth_month#231 IN (9,5,12,4,1,10) AND isnotnull(c_customer_sk#228)) AND isnotnull(c_current_cdemo_sk#229)) AND isnotnull(c_current_addr_sk#230)) (115) Project [codegen id : 46] -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] +Output [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_month#231, c_birth_year#232] (116) Scan parquet default.customer_address -Output [2]: [ca_address_sk#30, ca_state#32] +Output [2]: [ca_address_sk#233, ca_state#234] 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#30, ca_state#32] +Input [2]: [ca_address_sk#233, ca_state#234] (118) Filter [codegen id : 45] -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)) +Input [2]: [ca_address_sk#233, ca_state#234] +Condition : (ca_state#234 IN (ND,WI,AL,NC,OK,MS,TN) AND isnotnull(ca_address_sk#233)) (119) Project [codegen id : 45] -Output [1]: [ca_address_sk#30] -Input [2]: [ca_address_sk#30, ca_state#32] +Output [1]: [ca_address_sk#233] +Input [2]: [ca_address_sk#233, ca_state#234] (120) BroadcastExchange -Input [1]: [ca_address_sk#30] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#183] +Input [1]: [ca_address_sk#233] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#235] (121) BroadcastHashJoin [codegen id : 46] -Left keys [1]: [c_current_addr_sk#24] -Right keys [1]: [ca_address_sk#30] +Left keys [1]: [c_current_addr_sk#230] +Right keys [1]: [ca_address_sk#233] 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#30] +Output [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Input [5]: [c_customer_sk#228, c_current_cdemo_sk#229, c_current_addr_sk#230, c_birth_year#232, ca_address_sk#233] (123) BroadcastExchange -Input [3]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#184] +Input [3]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[1, int, true] as bigint)),false), [id=#236] (124) Scan parquet default.customer_demographics -Output [1]: [cd_demo_sk#185] +Output [1]: [cd_demo_sk#237] Batched: true Location [not included in comparison]/{warehouse_dir}/customer_demographics] PushedFilters: [IsNotNull(cd_demo_sk)] ReadSchema: struct (125) ColumnarToRow -Input [1]: [cd_demo_sk#185] +Input [1]: [cd_demo_sk#237] (126) Filter -Input [1]: [cd_demo_sk#185] -Condition : isnotnull(cd_demo_sk#185) +Input [1]: [cd_demo_sk#237] +Condition : isnotnull(cd_demo_sk#237) (127) BroadcastHashJoin [codegen id : 47] -Left keys [1]: [c_current_cdemo_sk#23] -Right keys [1]: [cd_demo_sk#185] +Left keys [1]: [c_current_cdemo_sk#229] +Right keys [1]: [cd_demo_sk#237] Join condition: None (128) Project [codegen id : 47] -Output [2]: [c_customer_sk#22, c_birth_year#26] -Input [4]: [c_customer_sk#22, c_current_cdemo_sk#23, c_birth_year#26, cd_demo_sk#185] +Output [2]: [c_customer_sk#228, c_birth_year#232] +Input [4]: [c_customer_sk#228, c_current_cdemo_sk#229, c_birth_year#232, cd_demo_sk#237] (129) BroadcastExchange -Input [2]: [c_customer_sk#22, c_birth_year#26] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#186] +Input [2]: [c_customer_sk#228, c_birth_year#232] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#238] (130) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#216] +Right keys [1]: [c_customer_sk#228] Join condition: None (131) Project [codegen id : 49] -Output [8]: [cs_item_sk#4, 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] -Input [10]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, c_customer_sk#22, c_birth_year#26] +Output [8]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232] +Input [10]: [cs_bill_customer_sk#216, cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_customer_sk#228, c_birth_year#232] (132) ReusedExchange [Reuses operator id: 21] -Output [2]: [i_item_sk#18, i_item_id#19] +Output [2]: [i_item_sk#239, i_item_id#240] (133) BroadcastHashJoin [codegen id : 49] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#218] +Right keys [1]: [i_item_sk#239] Join condition: None (134) Project [codegen id : 49] -Output [8]: [i_item_id#19, 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 [10]: [cs_item_sk#4, 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_sk#18, i_item_id#19] +Output [8]: [i_item_id#240, cast(cs_quantity#219 as decimal(12,2)) AS agg1#37, cast(cs_list_price#220 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#222 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#221 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#223 as decimal(12,2)) AS agg5#41, cast(c_birth_year#232 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#226 as decimal(12,2)) AS agg7#43] +Input [10]: [cs_item_sk#218, cs_quantity#219, cs_list_price#220, cs_sales_price#221, cs_coupon_amt#222, cs_net_profit#223, cd_dep_count#226, c_birth_year#232, i_item_sk#239, i_item_id#240] (135) HashAggregate [codegen id : 49] -Input [8]: [i_item_id#19, agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] -Keys [1]: [i_item_id#19] -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#187, count#188, sum#189, count#190, sum#191, count#192, sum#193, count#194, sum#195, count#196, sum#197, count#198, sum#199, count#200] -Results [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] +Input [8]: [i_item_id#240, agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] +Keys [1]: [i_item_id#240] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#241, count#242, sum#243, count#244, sum#245, count#246, sum#247, count#248, sum#249, count#250, sum#251, count#252, sum#253, count#254] +Results [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] (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), ENSURE_REQUIREMENTS, [id=#215] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Arguments: hashpartitioning(i_item_id#240, 5), ENSURE_REQUIREMENTS, [id=#269] (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] -Keys [1]: [i_item_id#19] -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)#216, avg(agg2#37)#217, avg(agg3#38)#218, avg(agg4#39)#219, avg(agg5#40)#220, avg(agg6#41)#221, avg(agg7#42)#222] -Results [11]: [i_item_id#19, null AS ca_country#223, null AS ca_state#224, null AS county#225, avg(agg1#36)#216 AS agg1#226, avg(agg2#37)#217 AS agg2#227, avg(agg3#38)#218 AS agg3#228, avg(agg4#39)#219 AS agg4#229, avg(agg5#40)#220 AS agg5#230, avg(agg6#41)#221 AS agg6#231, avg(agg7#42)#222 AS agg7#232] +Input [15]: [i_item_id#240, sum#255, count#256, sum#257, count#258, sum#259, count#260, sum#261, count#262, sum#263, count#264, sum#265, count#266, sum#267, count#268] +Keys [1]: [i_item_id#240] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#270, avg(agg2#38)#271, avg(agg3#39)#272, avg(agg4#40)#273, avg(agg5#41)#274, avg(agg6#42)#275, avg(agg7#43)#276] +Results [11]: [i_item_id#240, null AS ca_country#277, null AS ca_state#278, null AS county#279, avg(agg1#37)#270 AS agg1#280, avg(agg2#38)#271 AS agg2#281, avg(agg3#39)#272 AS agg3#282, avg(agg4#40)#273 AS agg4#283, avg(agg5#41)#274 AS agg5#284, avg(agg6#42)#275 AS agg6#285, avg(agg7#43)#276 AS agg7#286] (138) 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] +Output [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] Batched: true -Location [not included in comparison]/{warehouse_dir}/catalog_sales] -PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_sold_date_sk), IsNotNull(cs_item_sk)] -ReadSchema: struct +Location: InMemoryFileIndex [] +PartitionFilters: [isnotnull(cs_sold_date_sk#295), dynamicpruningexpression(cs_sold_date_sk#295 IN dynamicpruning#10)] +PushedFilters: [IsNotNull(cs_bill_cdemo_sk), IsNotNull(cs_bill_customer_sk), IsNotNull(cs_item_sk)] +ReadSchema: struct (139) ColumnarToRow [codegen id : 57] -Input [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] +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] (140) Filter [codegen id : 57] -Input [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] -Condition : (((isnotnull(cs_bill_cdemo_sk#3) AND isnotnull(cs_bill_customer_sk#2)) AND isnotnull(cs_sold_date_sk#1)) AND isnotnull(cs_item_sk#4)) +Input [9]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295] +Condition : ((isnotnull(cs_bill_cdemo_sk#288) AND isnotnull(cs_bill_customer_sk#287)) AND isnotnull(cs_item_sk#289)) (141) ReusedExchange [Reuses operator id: 8] -Output [2]: [cd_demo_sk#10, cd_dep_count#13] +Output [2]: [cd_demo_sk#296, cd_dep_count#297] (142) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_cdemo_sk#3] -Right keys [1]: [cd_demo_sk#10] +Left keys [1]: [cs_bill_cdemo_sk#288] +Right keys [1]: [cd_demo_sk#296] Join condition: None (143) Project [codegen id : 57] -Output [9]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [11]: [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, cd_demo_sk#10, cd_dep_count#13] +Output [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297] +Input [11]: [cs_bill_customer_sk#287, cs_bill_cdemo_sk#288, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_demo_sk#296, cd_dep_count#297] (144) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#15] +Output [1]: [d_date_sk#298] (145) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#15] +Left keys [1]: [cs_sold_date_sk#295] +Right keys [1]: [d_date_sk#298] Join condition: None (146) Project [codegen id : 57] -Output [8]: [cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13] -Input [10]: [cs_sold_date_sk#1, cs_bill_customer_sk#2, cs_item_sk#4, cs_quantity#5, cs_list_price#6, cs_sales_price#7, cs_coupon_amt#8, cs_net_profit#9, cd_dep_count#13, d_date_sk#15] +Output [8]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [10]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cs_sold_date_sk#295, cd_dep_count#297, d_date_sk#298] (147) Scan parquet default.item -Output [1]: [i_item_sk#18] +Output [1]: [i_item_sk#299] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct (148) ColumnarToRow [codegen id : 53] -Input [1]: [i_item_sk#18] +Input [1]: [i_item_sk#299] (149) Filter [codegen id : 53] -Input [1]: [i_item_sk#18] -Condition : isnotnull(i_item_sk#18) +Input [1]: [i_item_sk#299] +Condition : isnotnull(i_item_sk#299) (150) BroadcastExchange -Input [1]: [i_item_sk#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#233] +Input [1]: [i_item_sk#299] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [id=#300] (151) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_item_sk#4] -Right keys [1]: [i_item_sk#18] +Left keys [1]: [cs_item_sk#289] +Right keys [1]: [i_item_sk#299] Join condition: None (152) Project [codegen id : 57] -Output [7]: [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] -Input [9]: [cs_bill_customer_sk#2, cs_item_sk#4, 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_sk#18] +Output [7]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297] +Input [9]: [cs_bill_customer_sk#287, cs_item_sk#289, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, i_item_sk#299] (153) ReusedExchange [Reuses operator id: 129] -Output [2]: [c_customer_sk#22, c_birth_year#26] +Output [2]: [c_customer_sk#301, c_birth_year#302] (154) BroadcastHashJoin [codegen id : 57] -Left keys [1]: [cs_bill_customer_sk#2] -Right keys [1]: [c_customer_sk#22] +Left keys [1]: [cs_bill_customer_sk#287] +Right keys [1]: [c_customer_sk#301] Join condition: None (155) Project [codegen id : 57] -Output [7]: [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 [9]: [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, c_customer_sk#22, c_birth_year#26] +Output [7]: [cast(cs_quantity#290 as decimal(12,2)) AS agg1#37, cast(cs_list_price#291 as decimal(12,2)) AS agg2#38, cast(cs_coupon_amt#293 as decimal(12,2)) AS agg3#39, cast(cs_sales_price#292 as decimal(12,2)) AS agg4#40, cast(cs_net_profit#294 as decimal(12,2)) AS agg5#41, cast(c_birth_year#302 as decimal(12,2)) AS agg6#42, cast(cd_dep_count#297 as decimal(12,2)) AS agg7#43] +Input [9]: [cs_bill_customer_sk#287, cs_quantity#290, cs_list_price#291, cs_sales_price#292, cs_coupon_amt#293, cs_net_profit#294, cd_dep_count#297, c_customer_sk#301, c_birth_year#302] (156) HashAggregate [codegen id : 57] -Input [7]: [agg1#36, agg2#37, agg3#38, agg4#39, agg5#40, agg6#41, agg7#42] +Input [7]: [agg1#37, agg2#38, agg3#39, agg4#40, agg5#41, agg6#42, agg7#43] Keys: [] -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#234, count#235, sum#236, count#237, sum#238, count#239, sum#240, count#241, sum#242, count#243, sum#244, count#245, sum#246, count#247] -Results [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] +Functions [7]: [partial_avg(agg1#37), partial_avg(agg2#38), partial_avg(agg3#39), partial_avg(agg4#40), partial_avg(agg5#41), partial_avg(agg6#42), partial_avg(agg7#43)] +Aggregate Attributes [14]: [sum#303, count#304, sum#305, count#306, sum#307, count#308, sum#309, count#310, sum#311, count#312, sum#313, count#314, sum#315, count#316] +Results [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] (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, ENSURE_REQUIREMENTS, [id=#262] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, [id=#331] (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] +Input [14]: [sum#317, count#318, sum#319, count#320, sum#321, count#322, sum#323, count#324, sum#325, count#326, sum#327, count#328, sum#329, count#330] Keys: [] -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)#263, avg(agg2#37)#264, avg(agg3#38)#265, avg(agg4#39)#266, avg(agg5#40)#267, avg(agg6#41)#268, avg(agg7#42)#269] -Results [11]: [null AS i_item_id#270, null AS ca_country#271, null AS ca_state#272, null AS county#273, avg(agg1#36)#263 AS agg1#274, avg(agg2#37)#264 AS agg2#275, avg(agg3#38)#265 AS agg3#276, avg(agg4#39)#266 AS agg4#277, avg(agg5#40)#267 AS agg5#278, avg(agg6#41)#268 AS agg6#279, avg(agg7#42)#269 AS agg7#280] +Functions [7]: [avg(agg1#37), avg(agg2#38), avg(agg3#39), avg(agg4#40), avg(agg5#41), avg(agg6#42), avg(agg7#43)] +Aggregate Attributes [7]: [avg(agg1#37)#332, avg(agg2#38)#333, avg(agg3#39)#334, avg(agg4#40)#335, avg(agg5#41)#336, avg(agg6#42)#337, avg(agg7#43)#338] +Results [11]: [null AS i_item_id#339, null AS ca_country#340, null AS ca_state#341, null AS county#342, avg(agg1#37)#332 AS agg1#343, avg(agg2#38)#333 AS agg2#344, avg(agg3#39)#334 AS agg3#345, avg(agg4#40)#335 AS agg4#346, avg(agg5#41)#336 AS agg5#347, avg(agg6#42)#337 AS agg6#348, avg(agg7#43)#338 AS agg7#349] (159) Union (160) TakeOrderedAndProject -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] +Input [11]: [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] +Arguments: 100, [ca_country#31 ASC NULLS FIRST, ca_state#30 ASC NULLS FIRST, ca_county#29 ASC NULLS FIRST, i_item_id#20 ASC NULLS FIRST], [i_item_id#20, ca_country#31, ca_state#30, ca_county#29, agg1#80, agg2#81, agg3#82, agg4#83, agg5#84, agg6#85, agg7#86] + +===== Subqueries ===== + +Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 +ReusedExchange (161) + + +(161) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#16] + +Subquery:2 Hosting operator id = 103 Hosting Expression = cs_sold_date_sk#224 IN dynamicpruning#10 + +Subquery:3 Hosting operator id = 138 Hosting Expression = cs_sold_date_sk#295 IN dynamicpruning#10 + 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 4566929712713..5514e335f1b51 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] - 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) + 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] 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 - 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 + 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 - 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 (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] 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] #8 + ReusedExchange [cd_demo_sk] #9 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] #8 + ReusedExchange [cd_demo_sk] #9 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/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt index 15cde1a45a99b..3c2fc95d274b6 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt @@ -8,24 +8,24 @@ TakeOrderedAndProject (27) +- 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_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.catalog_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.catalog_sales @@ -35,118 +35,118 @@ Location [not included in comparison]/{warehouse_dir}/catalog_sales] PushedFilters: [IsNotNull(cs_item_sk), IsNotNull(cs_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] Condition : (isnotnull(cs_item_sk#2) AND isnotnull(cs_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: hashpartitioning(cs_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3] +Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [cs_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Input [4]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: hashpartitioning(cs_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [cs_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [cs_item_sk#2, cs_ext_sales_price#3] -Arguments: [cs_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [cs_sold_date_sk#1, cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [cs_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [cs_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [cs_item_sk#2, cs_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [cs_sold_date_sk#1, cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [cs_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [cs_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(cs_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) TakeOrderedAndProject -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: 100, [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt index e66efff75180a..3f5e15881c33c 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/simplified.txt @@ -14,34 +14,34 @@ TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_c WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,cs_ext_sales_price] [sum,sum] Project [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [cs_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [cs_item_sk] - InputAdapter - Exchange [cs_item_sk] #3 - WholeStageCodegen (2) - Project [cs_item_sk,cs_ext_sales_price] - BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + BroadcastHashJoin [cs_sold_date_sk,d_date_sk] + Project [cs_sold_date_sk,cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [cs_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [cs_item_sk] + InputAdapter + Exchange [cs_item_sk] #3 + WholeStageCodegen (1) Filter [cs_item_sk,cs_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.catalog_sales [cs_sold_date_sk,cs_item_sk,cs_ext_sales_price] - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #4 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #5 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #5 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] 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 04ff822b1ce52..a993f22e559ab 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 (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 (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 (58) : : +- Exchange (57) : : +- * Project (56) : : +- * BroadcastHashJoin Inner BuildRight (55) - : : :- * 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) + : : :- * Filter (50) + : : : +- * ColumnarToRow (49) + : : : +- Scan parquet default.inventory (48) : : +- BroadcastExchange (54) : : +- * Filter (53) : : +- * ColumnarToRow (52) @@ -98,7 +98,7 @@ Condition : (((((isnotnull(cs_quantity#8) AND isnotnull(cs_item_sk#5)) AND isnot 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), EqualTo(hd_buy_potential,1001-5000), IsNotNull(hd_demo_sk)] +PushedFilters: [IsNotNull(hd_buy_potential), EqualTo(hd_buy_potential,1001-5000 ), IsNotNull(hd_demo_sk)] ReadSchema: struct (5) ColumnarToRow [codegen id : 1] @@ -106,7 +106,7 @@ Input [2]: [hd_demo_sk#9, hd_buy_potential#10] (6) Filter [codegen id : 1] Input [2]: [hd_demo_sk#9, hd_buy_potential#10] -Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000)) AND isnotnull(hd_demo_sk#9)) +Condition : ((isnotnull(hd_buy_potential#10) AND (hd_buy_potential#10 = 1001-5000 )) AND isnotnull(hd_demo_sk#9)) (7) Project [codegen id : 1] Output [1]: [hd_demo_sk#9] @@ -213,95 +213,95 @@ Arguments: hashpartitioning(i_item_sk#19, 5), ENSURE_REQUIREMENTS, [id=#21] Input [2]: [i_item_sk#19, i_item_desc#20] Arguments: [i_item_sk#19 ASC NULLS FIRST], false, 0 -(31) SortMergeJoin [codegen id : 8] +(31) SortMergeJoin [codegen id : 10] Left keys [1]: [cs_item_sk#5] Right keys [1]: [i_item_sk#19] Join condition: None -(32) Project [codegen id : 8] +(32) Project [codegen id : 10] 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) 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] +(33) Scan parquet default.date_dim +Output [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] 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 -(36) ColumnarToRow [codegen id : 10] -Input [4]: [d_date_sk#23, d_date#24, d_week_seq#25, d_year#26] +(34) ColumnarToRow [codegen id : 8] +Input [4]: [d_date_sk#22, d_date#23, d_week_seq#24, d_year#25] -(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)) +(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)) -(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] +(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] -(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] +(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] -(40) Scan parquet default.date_dim -Output [2]: [d_date_sk#28, d_week_seq#29] +(38) Scan parquet default.date_dim +Output [2]: [d_date_sk#27, d_week_seq#28] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_week_seq), IsNotNull(d_date_sk)] ReadSchema: struct -(41) ColumnarToRow -Input [2]: [d_date_sk#28, d_week_seq#29] +(39) ColumnarToRow +Input [2]: [d_date_sk#27, d_week_seq#28] -(42) Filter -Input [2]: [d_date_sk#28, d_week_seq#29] -Condition : (isnotnull(d_week_seq#29) AND isnotnull(d_date_sk#28)) +(40) Filter +Input [2]: [d_date_sk#27, d_week_seq#28] +Condition : (isnotnull(d_week_seq#28) AND isnotnull(d_date_sk#27)) -(43) BroadcastHashJoin [codegen id : 11] -Left keys [1]: [d_week_seq#25] -Right keys [1]: [d_week_seq#29] +(41) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [d_week_seq#24] +Right keys [1]: [d_week_seq#28] Join condition: None -(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] +(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] -(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] +(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), ENSURE_REQUIREMENTS, [id=#30] -(46) Scan parquet default.inventory +(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 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 -(47) ColumnarToRow +(49) ColumnarToRow [codegen id : 13] Input [4]: [inv_date_sk#31, inv_item_sk#32, inv_warehouse_sk#33, inv_quantity_on_hand#34] -(48) Filter +(50) Filter [codegen id : 13] 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 [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] +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] (57) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#38] (58) Sort [codegen id : 14] -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 +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 (59) SortMergeJoin [codegen id : 16] -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)) +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) (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#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] +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] (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#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] +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] (67) Exchange -Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#25] +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), 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#25] +Input [5]: [cs_item_sk#5, cs_order_number#7, w_warehouse_name#36, i_item_desc#20, d_week_seq#24] Arguments: [cs_item_sk#5 ASC NULLS FIRST, cs_order_number#7 ASC NULLS FIRST], false, 0 (69) Scan parquet default.catalog_returns @@ -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#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] +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] (76) HashAggregate [codegen id : 20] -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] +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] Functions [1]: [partial_count(1)] Aggregate Attributes [1]: [count#45] -Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#25, count#46] +Results [4]: [i_item_desc#20, w_warehouse_name#36, d_week_seq#24, count#46] (77) Exchange -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] +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), ENSURE_REQUIREMENTS, [id=#47] (78) HashAggregate [codegen id : 21] -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] +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] Functions [1]: [count(1)] Aggregate Attributes [1]: [count(1)#48] -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] +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] (79) TakeOrderedAndProject -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] +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] 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 b88505ad7b9bc..918508787c4b0 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,cs_sold_date_sk,inv_item_sk,d_date_sk,inv_quantity_on_hand,cs_quantity,d_date,d_date] + SortMergeJoin [cs_item_sk,d_date_sk,inv_item_sk,inv_date_sk,inv_quantity_on_hand,cs_quantity] InputAdapter - WholeStageCodegen (9) - Sort [cs_item_sk,cs_sold_date_sk] + WholeStageCodegen (11) + Sort [cs_item_sk,d_date_sk] InputAdapter - 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 + 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] 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] + 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] InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [hd_demo_sk] - Filter [hd_buy_potential,hd_demo_sk] + BroadcastExchange #6 + WholeStageCodegen (2) + Project [cd_demo_sk] + Filter [cd_marital_status,cd_demo_sk] ColumnarToRow InputAdapter - Scan parquet default.household_demographics [hd_demo_sk,hd_buy_potential] + Scan parquet default.customer_demographics [cd_demo_sk,cd_marital_status] InputAdapter - 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] + 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] 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 #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] InputAdapter WholeStageCodegen (14) - Sort [inv_item_sk,d_date_sk] + Sort [inv_item_sk,inv_date_sk] InputAdapter - Exchange [inv_item_sk,d_date_sk] #9 + Exchange [inv_item_sk,inv_date_sk] #11 WholeStageCodegen (13) - Project [d_date_sk,d_date,d_week_seq,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] + Project [inv_date_sk,inv_item_sk,inv_quantity_on_hand,w_warehouse_name] BroadcastHashJoin [inv_warehouse_sk,w_warehouse_sk] - 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] + Filter [inv_quantity_on_hand,inv_item_sk,inv_warehouse_sk,inv_date_sk] + ColumnarToRow InputAdapter - 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] + 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-v2_7/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt index 6fa7b04a3b463..694b1ead8d0f0 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/explain.txt @@ -9,24 +9,24 @@ +- 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.store_sales (1) - : +- BroadcastExchange (8) - : +- * Project (7) - : +- * Filter (6) - : +- * ColumnarToRow (5) - : +- Scan parquet default.date_dim (4) - +- * Sort (17) - +- Exchange (16) + +- * BroadcastHashJoin Inner BuildRight (18) + :- * Project (12) + : +- * SortMergeJoin Inner (11) + : :- * Sort (5) + : : +- Exchange (4) + : : +- * Filter (3) + : : +- * ColumnarToRow (2) + : : +- Scan parquet default.store_sales (1) + : +- * Sort (10) + : +- Exchange (9) + : +- * Filter (8) + : +- * ColumnarToRow (7) + : +- Scan parquet default.item (6) + +- BroadcastExchange (17) + +- * Project (16) +- * Filter (15) +- * ColumnarToRow (14) - +- Scan parquet default.item (13) + +- Scan parquet default.date_dim (13) (1) Scan parquet default.store_sales @@ -36,122 +36,122 @@ Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_item_sk), IsNotNull(ss_sold_date_sk)] ReadSchema: struct -(2) ColumnarToRow [codegen id : 2] +(2) ColumnarToRow [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] -(3) Filter [codegen id : 2] +(3) Filter [codegen id : 1] Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] Condition : (isnotnull(ss_item_sk#2) AND isnotnull(ss_sold_date_sk#1)) -(4) Scan parquet default.date_dim -Output [2]: [d_date_sk#4, d_date#5] -Batched: true -Location [not included in comparison]/{warehouse_dir}/date_dim] -PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] -ReadSchema: struct +(4) Exchange +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: hashpartitioning(ss_item_sk#2, 5), ENSURE_REQUIREMENTS, [id=#4] -(5) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] +(5) Sort [codegen id : 2] +Input [3]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3] +Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 -(6) Filter [codegen id : 1] -Input [2]: [d_date_sk#4, d_date#5] -Condition : (((isnotnull(d_date#5) AND (d_date#5 >= 10644)) AND (d_date#5 <= 10674)) AND isnotnull(d_date_sk#4)) +(6) Scan parquet default.item +Output [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Batched: true +Location [not included in comparison]/{warehouse_dir}/item] +PushedFilters: [In(i_category, [Sports ,Books ,Home ]), IsNotNull(i_item_sk)] +ReadSchema: struct -(7) Project [codegen id : 1] -Output [1]: [d_date_sk#4] -Input [2]: [d_date_sk#4, d_date#5] +(7) ColumnarToRow [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(8) BroadcastExchange -Input [1]: [d_date_sk#4] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#6] +(8) Filter [codegen id : 3] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Condition : (i_category#10 IN (Sports ,Books ,Home ) AND isnotnull(i_item_sk#5)) -(9) BroadcastHashJoin [codegen id : 2] -Left keys [1]: [ss_sold_date_sk#1] -Right keys [1]: [d_date_sk#4] -Join condition: None +(9) Exchange +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: hashpartitioning(i_item_sk#5, 5), ENSURE_REQUIREMENTS, [id=#11] -(10) Project [codegen id : 2] -Output [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Input [4]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, d_date_sk#4] +(10) Sort [codegen id : 4] +Input [6]: [i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Arguments: [i_item_sk#5 ASC NULLS FIRST], false, 0 -(11) Exchange -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: hashpartitioning(ss_item_sk#2, 5), true, [id=#7] +(11) SortMergeJoin [codegen id : 6] +Left keys [1]: [ss_item_sk#2] +Right keys [1]: [i_item_sk#5] +Join condition: None -(12) Sort [codegen id : 3] -Input [2]: [ss_item_sk#2, ss_ext_sales_price#3] -Arguments: [ss_item_sk#2 ASC NULLS FIRST], false, 0 +(12) Project [codegen id : 6] +Output [7]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [9]: [ss_sold_date_sk#1, ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#5, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] -(13) Scan parquet default.item -Output [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(13) Scan parquet default.date_dim +Output [2]: [d_date_sk#12, d_date#13] Batched: true -Location [not included in comparison]/{warehouse_dir}/item] -PushedFilters: [In(i_category, [Sports,Books,Home]), IsNotNull(i_item_sk)] -ReadSchema: struct +Location [not included in comparison]/{warehouse_dir}/date_dim] +PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] +ReadSchema: struct -(14) ColumnarToRow [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +(14) ColumnarToRow [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] -(15) Filter [codegen id : 4] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Condition : (i_category#13 IN (Sports,Books,Home) AND isnotnull(i_item_sk#8)) +(15) Filter [codegen id : 5] +Input [2]: [d_date_sk#12, d_date#13] +Condition : (((isnotnull(d_date#13) AND (d_date#13 >= 10644)) AND (d_date#13 <= 10674)) AND isnotnull(d_date_sk#12)) -(16) Exchange -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: hashpartitioning(i_item_sk#8, 5), true, [id=#14] +(16) Project [codegen id : 5] +Output [1]: [d_date_sk#12] +Input [2]: [d_date_sk#12, d_date#13] -(17) Sort [codegen id : 5] -Input [6]: [i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Arguments: [i_item_sk#8 ASC NULLS FIRST], false, 0 +(17) BroadcastExchange +Input [1]: [d_date_sk#12] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [id=#14] -(18) SortMergeJoin [codegen id : 6] -Left keys [1]: [ss_item_sk#2] -Right keys [1]: [i_item_sk#8] +(18) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [ss_sold_date_sk#1] +Right keys [1]: [d_date_sk#12] Join condition: None (19) Project [codegen id : 6] -Output [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Input [8]: [ss_item_sk#2, ss_ext_sales_price#3, i_item_sk#8, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] +Output [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Input [8]: [ss_sold_date_sk#1, ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10, d_date_sk#12] (20) HashAggregate [codegen id : 6] -Input [6]: [ss_ext_sales_price#3, i_item_id#9, i_item_desc#10, i_current_price#11, i_class#12, i_category#13] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [ss_ext_sales_price#3, i_item_id#6, i_item_desc#7, i_current_price#8, i_class#9, i_category#10] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [partial_sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum#15] -Results [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] +Results [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] (21) Exchange -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Arguments: hashpartitioning(i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, 5), true, [id=#17] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Arguments: hashpartitioning(i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, 5), ENSURE_REQUIREMENTS, [id=#17] (22) HashAggregate [codegen id : 7] -Input [6]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, sum#16] -Keys [5]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11] +Input [6]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, sum#16] +Keys [5]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8] Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#3))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#3))#18] -Results [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] +Results [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS itemrevenue#19, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w0#20, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2) AS _w1#21] (23) Exchange -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: hashpartitioning(i_class#12, 5), true, [id=#22] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: hashpartitioning(i_class#9, 5), ENSURE_REQUIREMENTS, [id=#22] (24) Sort [codegen id : 8] -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [i_class#12 ASC NULLS FIRST], false, 0 +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [i_class#9 ASC NULLS FIRST], false, 0 (25) Window -Input [8]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21] -Arguments: [sum(_w1#21) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#12] +Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21] +Arguments: [sum(_w1#21) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#23], [i_class#9] (26) Project [codegen id : 9] -Output [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] -Input [9]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, _w0#20, _w1#21, _we0#23] +Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#20) * 100.00), DecimalType(21,2), true) as decimal(27,2))) / promote_precision(_we0#23)), DecimalType(38,17), true) AS revenueratio#24] +Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, _w0#20, _w1#21, _we0#23] (27) Exchange -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), true, [id=#25] +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: rangepartitioning(i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, [id=#25] (28) Sort [codegen id : 10] -Input [7]: [i_item_id#9, i_item_desc#10, i_category#13, i_class#12, i_current_price#11, itemrevenue#19, revenueratio#24] -Arguments: [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#9 ASC NULLS FIRST, i_item_desc#10 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 +Input [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#19, revenueratio#24] +Arguments: [i_category#10 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_item_id#6 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#24 ASC NULLS FIRST], true, 0 diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt index ded65becfe826..7066245b3973e 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.sf100/simplified.txt @@ -17,34 +17,34 @@ WholeStageCodegen (10) WholeStageCodegen (6) HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,ss_ext_sales_price] [sum,sum] Project [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - SortMergeJoin [ss_item_sk,i_item_sk] - InputAdapter - WholeStageCodegen (3) - Sort [ss_item_sk] - InputAdapter - Exchange [ss_item_sk] #4 - WholeStageCodegen (2) - Project [ss_item_sk,ss_ext_sales_price] - BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + BroadcastHashJoin [ss_sold_date_sk,d_date_sk] + Project [ss_sold_date_sk,ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + SortMergeJoin [ss_item_sk,i_item_sk] + InputAdapter + WholeStageCodegen (2) + Sort [ss_item_sk] + InputAdapter + Exchange [ss_item_sk] #4 + WholeStageCodegen (1) Filter [ss_item_sk,ss_sold_date_sk] ColumnarToRow InputAdapter Scan parquet default.store_sales [ss_sold_date_sk,ss_item_sk,ss_ext_sales_price] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (1) - Project [d_date_sk] - Filter [d_date,d_date_sk] - ColumnarToRow - InputAdapter - Scan parquet default.date_dim [d_date_sk,d_date] + InputAdapter + WholeStageCodegen (4) + Sort [i_item_sk] + InputAdapter + Exchange [i_item_sk] #5 + WholeStageCodegen (3) + Filter [i_category,i_item_sk] + ColumnarToRow + InputAdapter + Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] InputAdapter - WholeStageCodegen (5) - Sort [i_item_sk] - InputAdapter - Exchange [i_item_sk] #6 - WholeStageCodegen (4) - Filter [i_category,i_item_sk] - ColumnarToRow - InputAdapter - Scan parquet default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + BroadcastExchange #6 + WholeStageCodegen (5) + Project [d_date_sk] + Filter [d_date,d_date_sk] + ColumnarToRow + InputAdapter + Scan parquet default.date_dim [d_date_sk,d_date] diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q1.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q1.sql.out new file mode 100644 index 0000000000000..d3b808c48dc2d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q1.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAABAAAA +AAAAAAAAAAAHBAAA +AAAAAAAAAAAMAAAA +AAAAAAAAAAAOAAAA +AAAAAAAAAABCBAAA +AAAAAAAAAABEAAAA +AAAAAAAAAABFAAAA +AAAAAAAAAACFBAAA +AAAAAAAAAACFBAAA +AAAAAAAAAADBBAAA +AAAAAAAAAADOAAAA +AAAAAAAAAADPAAAA +AAAAAAAAAAEABAAA +AAAAAAAAAAEEAAAA +AAAAAAAAAAEGBAAA +AAAAAAAAAAENAAAA +AAAAAAAAAAFCBAAA +AAAAAAAAAAFEBAAA +AAAAAAAAAAFGAAAA +AAAAAAAAAAFLAAAA +AAAAAAAAAAFPAAAA +AAAAAAAAAAGCAAAA +AAAAAAAAAAGEAAAA +AAAAAAAAAAGIBAAA +AAAAAAAAAAGOAAAA +AAAAAAAAAAHABAAA +AAAAAAAAAAHGBAAA +AAAAAAAAAAHHAAAA +AAAAAAAAAAHMAAAA +AAAAAAAAAAHPAAAA +AAAAAAAAAAHPAAAA +AAAAAAAAAAHPAAAA +AAAAAAAAAAJAAAAA +AAAAAAAAAAJEBAAA +AAAAAAAAAAJMAAAA +AAAAAAAAAAJPAAAA +AAAAAAAAAAKBBAAA +AAAAAAAAAAKGBAAA +AAAAAAAAAAKHBAAA +AAAAAAAAAAKLAAAA +AAAAAAAAAALCAAAA +AAAAAAAAAALJAAAA +AAAAAAAAAALJAAAA +AAAAAAAAAAMABAAA +AAAAAAAAAAMGAAAA +AAAAAAAAAAMLAAAA +AAAAAAAAAAMMAAAA +AAAAAAAAAANHBAAA +AAAAAAAAAANMAAAA +AAAAAAAAAAOBBAAA +AAAAAAAAAAPDAAAA +AAAAAAAAAAPKAAAA +AAAAAAAAAAPLAAAA +AAAAAAAAABANAAAA +AAAAAAAAABCCBAAA +AAAAAAAAABCGAAAA +AAAAAAAAABDABAAA +AAAAAAAAABDBAAAA +AAAAAAAAABDEAAAA +AAAAAAAAABDEBAAA +AAAAAAAAABDEBAAA +AAAAAAAAABDFBAAA +AAAAAAAAABDOAAAA +AAAAAAAAABDOAAAA +AAAAAAAAABEBBAAA +AAAAAAAAABEDAAAA +AAAAAAAAABEEAAAA +AAAAAAAAABEEBAAA +AAAAAAAAABEIBAAA +AAAAAAAAABEOAAAA +AAAAAAAAABFFBAAA +AAAAAAAAABFHAAAA +AAAAAAAAABFNAAAA +AAAAAAAAABFOAAAA +AAAAAAAAABGAAAAA +AAAAAAAAABHDBAAA +AAAAAAAAABHGAAAA +AAAAAAAAABHGBAAA +AAAAAAAAABHLAAAA +AAAAAAAAABIAAAAA +AAAAAAAAABIBAAAA +AAAAAAAAABIDBAAA +AAAAAAAAABIEBAAA +AAAAAAAAABKLAAAA +AAAAAAAAABKNAAAA +AAAAAAAAABKNAAAA +AAAAAAAAABLJAAAA +AAAAAAAAABLNAAAA +AAAAAAAAABMAAAAA +AAAAAAAAABMEBAAA +AAAAAAAAABMPAAAA +AAAAAAAAABNABAAA +AAAAAAAAABNBAAAA +AAAAAAAAABNEAAAA +AAAAAAAAABNEAAAA +AAAAAAAAABNGAAAA +AAAAAAAAABNNAAAA +AAAAAAAAABOEAAAA +AAAAAAAAABOGBAAA +AAAAAAAAABPABAAA diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q10.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q10.sql.out new file mode 100644 index 0000000000000..195c969028573 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q10.sql.out @@ -0,0 +1,11 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +F W 4 yr Degree 1 4000 1 Low Risk 1 4 1 6 1 4 1 +M D 4 yr Degree 1 1500 1 Low Risk 1 3 1 4 1 2 1 +M S College 1 4500 1 High Risk 1 3 1 4 1 3 1 +M S Primary 1 9500 1 Low Risk 1 3 1 0 1 6 1 +M S Secondary 1 3000 1 High Risk 1 1 1 1 1 4 1 +M U 4 yr Degree 1 2000 1 Low Risk 1 3 1 1 1 3 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q11.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q11.sql.out new file mode 100644 index 0000000000000..8e02562680cf9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q11.sql.out @@ -0,0 +1,99 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL +NULL +NULL +NULL +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +N +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y +Y diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q12.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q12.sql.out new file mode 100644 index 0000000000000..92e6e202b7a33 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q12.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Precisely elderly bodies Books arts 1.40 11.21 0.01417562243122168 +Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 515.52 0.65190159462474560 +Forward psychological plants establish closely yet eastern changes. Likewise necessary techniques might drop. Pleasant operations like lonely things; dogs let regions. Forces might not result clearl Books arts 2.43 11462.46 14.49487110552909973 +Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 3400.60 4.30023386615632740 +United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 784.30 0.99178774958137022 +Naturally new years put serious, negative vehicles. Fin Books arts 3.34 3319.96 4.19826043236027781 +Hard different differences would not paint even. Together suitable schemes marry directly only open women. Social ca Books arts 2.65 229.68 0.29044219090124839 +Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 5752.44 7.27425669029944833 +Powerful walls will find; there scottish decades must not Books arts 4.16 434.76 0.54977641464745189 +Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 969.48 1.22595739827125692 +Aside legitimate decisions may not stand probably sexual g Books arts 3.88 349.20 0.44158138742039332 +Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 3343.99 4.22864760515441312 +Unlikely states take later in general extra inf Books arts 0.32 20046.98 25.35043883731064290 +Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 11869.78 15.00994822673206253 +Relevant lips take so sure, manufacturing Books arts 8.80 5995.28 7.58134037907713537 +Extra, primitive weeks look obviou Books arts 1.18 425.89 0.53855984275049058 +More than key reasons should remain. Words used to offer slowly british Books arts 0.28 7814.52 9.88186306879843074 +Children may turn also above, historical aspects. Surveys migh Books arts 7.22 544.72 0.68882649872748182 +Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 152.67 0.19305907908783347 +Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 503.10 0.63619586486597904 +Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 922.40 1.16642231316314662 +Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 230.48 0.29145383210954253 +New, reluctant associations see more different, physical symptoms; useful pounds ought to give. Subjects Books business 9.02 306.85 0.37352072221391094 +Natural plans might not like n Books business 4.29 2484.54 3.02436752540117416 +Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 90.28 0.10989555418436330 +Simple, great shops glance from a years. Lessons deepen here previous clients. Increased, silent flights open more great rocks. Brill Books business 8.92 393.75 0.47930188812686144 +Groups must not put new, civil moves. Correct men laugh slightly total novels. Relatively public girls set even scott Books business 3.36 344.10 0.41886420242400767 +Just young degrees stop posts. More than tight artists buy to a arts. European, essential techniques ought to sell to a offences. Sentences be Books business 2.58 184.08 0.22407591508925118 +Junior, severe restrictions ought to want principles. Sure, Books business 9.77 1549.80 1.88653223166732663 +Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 6463.45 7.86779374936777799 +Russian windows should see in a weapons. New, considerable branches walk. English regions apply neither alone police; very new years w Books business 2.79 1635.60 1.99097439548011320 +Long groups used to create more tiny feet; tools used to dare still Books business 57.04 10558.62 12.85274032257534413 +Drugs must compensate dark, modest houses. Small pubs claim naturally accessible relationships. Distinguished Books business 1.66 31.78 0.03868498794837246 +Small, capable centres Books business 2.98 3219.72 3.91928349267255446 +Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 3862.88 4.70218584789203943 +Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 10904.74 13.27406341976510738 +Sound, original activities consider quite to a attitudes. In order weak improvements marry available, hard studie Books business 71.27 385.84 0.46967324575204627 +Classic issues will draw as european, engl Books business 75.64 92.64 0.11276832232653319 +Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 5353.50 6.51667976657054660 +Departments could seek now for a commu Books business 5.93 6535.44 7.95542535045032467 +Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 303.68 0.36966196161616580 +Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 2096.55 2.55207713918062566 +Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 6646.96 8.09117581791421695 +White fees might combine reports. Tr Books business 2.09 500.56 0.60931899205277908 +Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 974.26 1.18594198736882801 +Only new systems might join late speeches. Materials could stay on a benefits. Corporate regulations must crawl definitely practical deaths. Windows might soothe despite a organisations. Old Books business 0.67 9075.35 11.04719337247520503 +Professional managers form later initial grounds. Conscious, big risks restore. American, full rises say for a systems. Already Books business 5.27 890.13 1.08353267219901759 +Memories can earn particularly over quick contexts; alone differences make separate years; irish men mea Books business 4.23 2059.92 2.50748836924516678 +Only, gothic Books business 1.68 4777.17 5.81512787530920297 +Silver, critical operations could help howev Books business 5.56 428.54 0.52165087273113702 +Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 2456.28 1.26602601850774711 +At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 10252.90 5.28459221471415324 +Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 1761.68 0.90801240749618444 +Ga Books computers 5.53 7541.48 3.88706087988983530 +S Books computers 65.78 4566.02 2.35343695385979752 +Boxes batt Books computers 0.83 7844.04 4.04300760915510798 +Artists make times. Rather ready functions must pre Books computers 5.71 3694.01 1.90398194531071494 +Limited, capable cities shall try during the bodies. Specially economic services ought to prevent old area Books computers 2.93 96.18 0.04957349425150028 +Legs throw then. Old-fashioned develo Books computers 2.66 718.55 0.37035801928067716 +Important, educational variables used to appear months. A Books computers 2.47 9922.02 5.11404867366677942 +Men should not turn shadows. Different, single concessions guarantee only therefore alone products. Books computers 8.38 4194.24 2.16181256528813215 +Educational, white teachers should not fix. Considerable, other services might not cover today on a forms. Successful genes fall otherwise so Books computers 1.65 14569.68 7.50956485471198434 +Present Books computers 2.84 12393.53 6.38792460190056468 +Multiple, dark feet mean more complex girls; schools may not answer frequently blue assets. Spiritual, dry patients may reply personnel Books computers 2.04 371.40 0.19142852739662305 +Private teachers ap Books computers 5.27 4911.39 2.53144899076602182 +Daily numbers sense interesting players. General advantages would speak here. Shelves shall know with the reductions. Again wrong mothers provide ways; as hot pr Books computers 7.56 689.26 0.35526124607807325 +Inc, corporate ships slow evident degrees. Chosen, acute prices throw always. Budgets spend points. Commonly large events may mean. Bottles c Books computers 68.38 4.17 0.00214931868401701 +However old hours ma Books computers 8.84 451.53 0.23272946412330966 +Indeed other actions should provide after a ideas; exhibitio Books computers 6.95 8062.32 4.15551439149257400 +Perfect days find at all. Crimes might develop hopes. Much socialist grants drive current, useful walls. Emissions open naturally. Combinations shall not know. Tragic things shall not receive just Books computers 6.71 526.49 0.27136565802113105 +Human windows take right, variable steps. Years should buy often. Indeed thin figures may beat even up to a cars. Details may tell enough. Impossible, sufficient differences ought to return Books computers 4.47 1542.60 0.79509328584283986 +Left diff Books computers 0.74 5248.81 2.70536340572070289 +Friendly, hot computers tax elsewhere units. New, real officials should l Books computers 3.19 12378.72 6.38029117031536278 +Kinds relieve really major practices. Then capable reserves could not approve foundations. Pos Books computers 7.23 1786.41 0.92075884659828053 +Visible, average words shall not want quite only public boundaries. Cars must telephone proposals. German things ask abilities. Windows cut again favorite offi Books computers 6.75 25255.89 13.01749550563031296 +Only increased errors must submit as rich, main Books computers 6.94 2429.79 1.25237243291071818 +Meals ought to test. Round days might need most urban years. Political, english pages must see on a eyes. Only subsequent women may come better methods; difficult, social childr Books computers 7.23 6457.72 3.32846480866914548 +Systems cannot see fairly practitioners. Little ca Books computers 1.73 6197.59 3.19438752586978211 +Past beautiful others might not like more than legislative, small products. Close, wh Books computers 3.02 10232.30 5.27397447733028024 +Main problems wait properly. Everyday, foreign offenders can worry activities. Social, important shoes will afford okay physical parts. Very Books computers 1.40 2034.30 1.04852733786470188 +Schools offer quickly others. Further main buildings satisfy sadly great, productive figures. Years contribute acti Books computers 4.11 3975.90 2.04927485750197523 +Tiny, rare leaders mention old, precious areas; students will improve much multiple stars. Even confident solutions will include clearly single women. Please little rights will not mention harder com Books computers 1.45 11902.91 6.13504720795513872 +Guidelines should investigate so. Usual personnel look now old, modern aspects. Discussions could appear once br Books computers 2.44 640.06 0.32990237815154161 +Flat pleasant groups would go private, redundant eyes. Main devic Books computers 2.83 8359.41 4.30864175068552700 +Fine users go for a networks. Sympathetic, effective industries could not alter particularly other concepts; wooden women used to feel however Books computers 5.30 247.79 0.12771694885193653 +Real, domestic facilities turn often guilty symptoms. Winds get naturally intense islands. Products shall not travel a little clear shares; improved children may not apply wrong c Books computers 5.28 297.60 0.15339022550682558 +Then irish champions must look now forward good women. Future, big models sign. Then different o Books cooking 85.81 6496.48 10.66582432143788856 +Valuable studies should persist so concerned parties. Always polite songs include then from the holes. There conventional areas might not explain theore Books cooking 1.58 2088.03 3.42809662430915734 +Meanings occur in a things. Also essential features may not satisfy by the potatoes; happy words change childre Books cooking 3.46 1496.40 2.45676728237440221 +Then dominant goods should combine probably american items. Important artists guess only sill Books cooking 6.67 5638.06 9.25648312220250073 +Individual, remarkable services take by the interest Books cooking 6.05 0.00 0.00000000000000000 +Ultimately senior elections marry at l Books cooking 5.06 2060.48 3.38286544372280691 +Hence young effects shall not solve however months. In order small activities must not return almost national foods. International decades take contributions. Sessions must see Books cooking 1.43 242.28 0.39777170353760369 +Points trace so simple eyes. Short advisers shall not say limitations. Keys stretch in full now blue wings. Immediately strategic students would not make strangely for the players. Books cooking 1.69 12518.00 20.55186637313737424 +Great pp. will not r Books cooking 1.91 7268.22 11.93285558480304571 +Products may not resist further specif Books cooking 5.37 2.72 0.00446565557876128 +Somet Books cooking 7.34 580.58 0.95318761614603744 +Genetic properties might describe therefore leaves; right other organisers must not talk even lives; methods carry thus wrong minutes. Proud worke Books cooking 1.08 1445.15 2.37262579398781566 +Urgent agencies mean over as a plants; then Books cooking 6.47 1312.79 2.15531911295662354 +As light hundreds must establish on a services. Sometimes special results Books cooking 44.82 3513.30 5.76808372972867366 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q13.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q13.sql.out new file mode 100644 index 0000000000000..518f598f30aa6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q13.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +31.875 2306.480000 2168.643750 17349.15 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q14a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q14a.sql.out new file mode 100644 index 0000000000000..ae4502a8cd93c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q14a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL 674681657.38 155667 +catalog NULL NULL NULL 235666845.52 46085 +catalog 1001001 NULL NULL 1698135.15 341 +catalog 1001001 1 NULL 1050210.86 214 +catalog 1001001 1 1 42507.62 8 +catalog 1001001 1 2 106083.96 19 +catalog 1001001 1 3 101297.49 22 +catalog 1001001 1 4 68779.51 25 +catalog 1001001 1 5 201671.76 31 +catalog 1001001 1 6 165473.64 25 +catalog 1001001 1 7 57452.17 9 +catalog 1001001 1 8 107124.01 25 +catalog 1001001 1 9 61121.32 18 +catalog 1001001 1 10 138699.38 32 +catalog 1001001 2 NULL 34937.77 4 +catalog 1001001 2 3 34937.77 4 +catalog 1001001 3 NULL 208126.74 43 +catalog 1001001 3 1 65982.89 13 +catalog 1001001 3 4 108943.96 20 +catalog 1001001 3 6 23652.37 7 +catalog 1001001 3 7 9547.52 3 +catalog 1001001 4 NULL 20486.85 5 +catalog 1001001 4 7 20486.85 5 +catalog 1001001 5 NULL 41532.27 8 +catalog 1001001 5 7 11217.24 4 +catalog 1001001 5 9 30315.03 4 +catalog 1001001 8 NULL 139099.45 25 +catalog 1001001 8 9 77814.57 15 +catalog 1001001 8 10 61284.88 10 +catalog 1001001 10 NULL 24174.96 5 +catalog 1001001 10 7 24174.96 5 +catalog 1001001 13 NULL 20891.29 3 +catalog 1001001 13 9 20891.29 3 +catalog 1001001 14 NULL 66038.13 14 +catalog 1001001 14 9 26363.83 7 +catalog 1001001 14 10 39674.30 7 +catalog 1001001 15 NULL 63699.54 12 +catalog 1001001 15 10 63699.54 12 +catalog 1001001 16 NULL 28937.29 8 +catalog 1001001 16 9 28937.29 8 +catalog 1001002 NULL NULL 3137545.73 657 +catalog 1001002 1 NULL 2597194.10 539 +catalog 1001002 1 1 2597194.10 539 +catalog 1001002 2 NULL 120152.76 28 +catalog 1001002 2 1 120152.76 28 +catalog 1001002 3 NULL 43520.49 11 +catalog 1001002 3 1 43520.49 11 +catalog 1001002 4 NULL 138778.24 30 +catalog 1001002 4 1 138778.24 30 +catalog 1001002 7 NULL 26096.94 4 +catalog 1001002 7 1 26096.94 4 +catalog 1001002 10 NULL 15192.85 6 +catalog 1001002 10 1 15192.85 6 +catalog 1001002 12 NULL 79002.90 17 +catalog 1001002 12 1 79002.90 17 +catalog 1001002 13 NULL 28858.71 5 +catalog 1001002 13 1 28858.71 5 +catalog 1001002 14 NULL 88748.74 17 +catalog 1001002 14 1 88748.74 17 +catalog 1002001 NULL NULL 1812816.55 357 +catalog 1002001 1 NULL 158520.06 30 +catalog 1002001 1 1 25188.03 6 +catalog 1002001 1 2 77805.61 11 +catalog 1002001 1 4 15320.55 6 +catalog 1002001 1 5 40205.87 7 +catalog 1002001 2 NULL 944972.06 193 +catalog 1002001 2 1 69392.20 14 +catalog 1002001 2 2 143259.76 24 +catalog 1002001 2 3 190639.48 38 +catalog 1002001 2 4 60154.92 11 +catalog 1002001 2 6 72905.53 16 +catalog 1002001 2 7 17237.62 3 +catalog 1002001 2 8 141091.32 36 +catalog 1002001 2 9 141603.60 25 +catalog 1002001 2 10 108687.63 26 +catalog 1002001 3 NULL 61348.83 9 +catalog 1002001 3 1 61348.83 9 +catalog 1002001 4 NULL 126215.12 28 +catalog 1002001 4 1 59437.19 12 +catalog 1002001 4 3 24792.24 9 +catalog 1002001 4 5 41985.69 7 +catalog 1002001 5 NULL 60039.55 7 +catalog 1002001 5 9 60039.55 7 +catalog 1002001 6 NULL 160520.59 27 +catalog 1002001 6 7 10403.70 5 +catalog 1002001 6 9 100458.19 15 +catalog 1002001 6 10 49658.70 7 +catalog 1002001 8 NULL 18830.90 5 +catalog 1002001 8 8 18830.90 5 +catalog 1002001 9 NULL 31370.49 5 +catalog 1002001 9 9 31370.49 5 +catalog 1002001 10 NULL 40164.96 9 +catalog 1002001 10 6 7759.96 5 +catalog 1002001 10 8 32405.00 4 +catalog 1002001 11 NULL 42852.27 11 +catalog 1002001 11 8 25836.23 7 +catalog 1002001 11 9 17016.04 4 +catalog 1002001 12 NULL 12275.20 4 +catalog 1002001 12 7 12275.20 4 +catalog 1002001 13 NULL 27702.27 5 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q14b.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q14b.sql.out new file mode 100644 index 0000000000000..cf254c3d32fdf --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q14b.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +store 1001001 1 1 618811.76 171 store 1001001 1 1 1120886.77 328 +store 1001002 1 1 679257.16 207 store 1001002 1 1 597317.93 174 +store 1002001 2 1 590902.05 166 store 1002001 2 1 1382457.19 367 +store 1002002 2 1 712644.78 166 store 1002002 2 1 686649.72 155 +store 1003001 3 1 567349.84 161 store 1003001 3 1 1120432.56 309 +store 1003002 3 1 775782.67 214 store 1003002 3 1 561917.26 173 +store 1004001 4 1 602670.66 187 store 1004001 4 1 1284658.68 355 +store 1004002 4 1 680693.19 185 store 1004002 4 1 572656.37 181 +store 2001001 1 2 721151.24 193 store 2001001 1 2 1206198.64 359 +store 2001002 1 2 815659.11 223 store 2001002 1 2 621816.55 166 +store 2002001 2 2 657826.11 158 store 2002001 2 2 1229089.24 366 +store 2002002 2 2 738888.97 213 store 2002002 2 2 701020.25 187 +store 2003001 3 2 856103.36 229 store 2003001 3 2 1517919.93 408 +store 2003002 3 2 1035024.58 255 store 2003002 3 2 693873.11 187 +store 2004001 4 2 818535.48 210 store 2004001 4 2 1584158.70 423 +store 2004002 4 2 902066.26 231 store 2004002 4 2 652868.55 184 +store 3001001 1 3 690099.22 187 store 3001001 1 3 1540182.88 363 +store 3001002 1 3 796955.01 238 store 3001002 1 3 690610.81 178 +store 3002001 2 3 601571.31 163 store 3002001 2 3 1332847.53 353 +store 3002002 2 3 889691.45 239 store 3002002 2 3 711790.52 187 +store 3003001 3 3 599641.77 168 store 3003001 3 3 1218300.45 329 +store 3003002 3 3 785846.50 222 store 3003002 3 3 727813.84 205 +store 3004001 4 3 636017.18 169 store 3004001 4 3 1300058.21 347 +store 3004002 4 3 850298.30 240 store 3004002 4 3 605722.91 192 +store 4001001 1 4 709294.08 189 store 4001001 1 4 1317772.16 332 +store 4001002 1 4 833503.02 203 store 4001002 1 4 651619.28 161 +store 4002001 2 4 633353.59 188 store 4002001 2 4 1235025.47 341 +store 4002002 2 4 928549.29 239 store 4002002 2 4 724613.68 193 +store 4003001 3 4 645858.10 173 store 4003001 3 4 1417068.83 390 +store 4003002 3 4 872089.82 236 store 4003002 3 4 707566.93 201 +store 4004001 4 4 657080.07 179 store 4004001 4 4 1279798.81 334 +store 4004002 4 4 836177.44 216 store 4004002 4 4 821961.41 198 +store 5001001 1 5 719353.50 186 store 5001001 1 5 1189108.84 310 +store 5001002 1 5 842595.01 234 store 5001002 1 5 788955.51 198 +store 5002001 2 5 773460.67 218 store 5002001 2 5 1274443.41 357 +store 5002002 2 5 910604.00 253 store 5002002 2 5 649585.02 178 +store 5003001 3 5 699927.22 177 store 5003001 3 5 1388232.27 389 +store 5003002 3 5 795797.08 221 store 5003002 3 5 640802.27 168 +store 5004001 4 5 746611.79 186 store 5004001 4 5 1149863.23 334 +store 5004002 4 5 661403.11 175 store 5004002 4 5 522457.04 139 +store 6001001 1 6 32952.14 10 store 6001001 1 6 100674.40 25 +store 6001002 1 6 32363.63 11 store 6001002 1 6 21192.44 6 +store 6001003 1 6 32105.72 12 store 6001003 1 6 49799.29 16 +store 6001004 1 6 69366.44 18 store 6001004 1 6 33159.67 12 +store 6001005 1 6 39316.28 13 store 6001005 1 6 176957.80 40 +store 6001006 1 6 33851.16 8 store 6001006 1 6 59928.72 18 +store 6001007 1 6 52934.26 10 store 6001007 1 6 68730.18 15 +store 6001008 1 6 61470.12 17 store 6001008 1 6 21110.81 7 +store 6002001 2 6 58059.46 12 store 6002001 2 6 118560.37 25 +store 6002002 2 6 45696.16 15 store 6002002 2 6 98188.35 25 +store 6002003 2 6 27662.18 10 store 6002003 2 6 52033.85 19 +store 6002004 2 6 49212.09 11 store 6002004 2 6 23087.83 7 +store 6002005 2 6 54618.54 15 store 6002005 2 6 94438.29 22 +store 6002006 2 6 4768.74 6 store 6002006 2 6 41696.39 7 +store 6002007 2 6 23765.55 6 store 6002007 2 6 100119.20 29 +store 6002008 2 6 26747.76 5 store 6002008 2 6 19965.51 3 +store 6003001 3 6 17874.97 4 store 6003001 3 6 39734.27 10 +store 6003004 3 6 36845.45 13 store 6003004 3 6 28332.73 5 +store 6003005 3 6 33876.00 12 store 6003005 3 6 149594.43 34 +store 6003006 3 6 78594.03 17 store 6003006 3 6 80057.92 16 +store 6003007 3 6 90969.04 23 store 6003007 3 6 130880.43 32 +store 6003008 3 6 51335.82 14 store 6003008 3 6 45068.89 16 +store 6004001 4 6 14673.04 4 store 6004001 4 6 84488.05 22 +store 6004002 4 6 41801.05 15 store 6004002 4 6 31527.39 12 +store 6004003 4 6 15735.99 8 store 6004003 4 6 102005.55 23 +store 6004004 4 6 70293.74 15 store 6004004 4 6 16694.72 7 +store 6004006 4 6 98506.28 26 store 6004006 4 6 40380.88 13 +store 6004007 4 6 7974.20 5 store 6004007 4 6 56390.64 19 +store 6004008 4 6 93726.06 22 store 6004008 4 6 49454.40 15 +store 6005001 5 6 25935.02 8 store 6005001 5 6 76459.38 18 +store 6005002 5 6 27560.91 8 store 6005002 5 6 29874.66 8 +store 6005003 5 6 76424.40 23 store 6005003 5 6 120380.82 37 +store 6005004 5 6 61026.67 15 store 6005004 5 6 32867.59 7 +store 6005005 5 6 49398.50 13 store 6005005 5 6 125117.31 33 +store 6005006 5 6 40539.82 9 store 6005006 5 6 24310.11 8 +store 6005007 5 6 18454.21 5 store 6005007 5 6 52454.65 14 +store 6005008 5 6 71116.83 19 store 6005008 5 6 40388.90 6 +store 6006001 6 6 26990.02 8 store 6006001 6 6 19931.96 5 +store 6006003 6 6 39807.40 6 store 6006003 6 6 100194.33 22 +store 6006004 6 6 126180.26 29 store 6006004 6 6 44060.41 16 +store 6006005 6 6 24214.85 8 store 6006005 6 6 79947.56 24 +store 6006006 6 6 59581.58 21 store 6006006 6 6 45979.99 11 +store 6006008 6 6 81635.42 14 store 6006008 6 6 26221.85 8 +store 6007001 7 6 15407.87 3 store 6007001 7 6 14185.50 7 +store 6007002 7 6 77223.28 17 store 6007002 7 6 21012.93 7 +store 6007003 7 6 30617.75 9 store 6007003 7 6 21503.75 7 +store 6007004 7 6 35752.37 11 store 6007004 7 6 39378.54 9 +store 6007005 7 6 49598.38 16 store 6007005 7 6 102247.83 29 +store 6007006 7 6 43099.21 12 store 6007006 7 6 17599.96 11 +store 6007007 7 6 36750.53 10 store 6007007 7 6 47280.71 19 +store 6007008 7 6 63116.82 21 store 6007008 7 6 16366.95 10 +store 6008001 8 6 78930.46 18 store 6008001 8 6 123392.71 36 +store 6008002 8 6 20148.09 5 store 6008002 8 6 98780.67 17 +store 6008004 8 6 41158.44 15 store 6008004 8 6 7371.54 4 +store 6008005 8 6 52094.82 11 store 6008005 8 6 153697.84 42 +store 6008006 8 6 40340.12 8 store 6008006 8 6 32564.87 11 +store 6008007 8 6 8554.42 2 store 6008007 8 6 40138.79 9 +store 6008008 8 6 26928.47 7 store 6008008 8 6 18485.16 4 +store 6009001 9 6 69811.04 16 store 6009001 9 6 136769.36 32 +store 6009002 9 6 31689.61 14 store 6009002 9 6 63253.41 19 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q15.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q15.sql.out new file mode 100644 index 0000000000000..7ec445691906f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q15.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL 2519.19 +30059 812.16 +30069 2248.42 +30150 157.03 +30162 2039.97 +30169 1670.46 +30191 544.76 +30399 638.05 +30411 771.09 +30499 1826.69 +30587 3142.51 +30945 159.37 +30965 331.74 +31087 1264.39 +31218 1221.13 +31289 366.49 +31387 1650.86 +31521 490.31 +31675 1909.58 +31711 458.28 +31749 1229.27 +31757 1207.80 +31904 1584.89 +31933 829.70 +31952 794.71 +32293 1025.43 +32477 95.37 +32808 1306.54 +32812 209.90 +32898 448.94 +33003 236.71 +33199 385.03 +33372 170.95 +33394 842.14 +33445 1084.78 +33511 447.67 +33604 616.64 +33951 2103.68 +33957 319.38 +34098 288.86 +34107 2903.74 +34167 717.59 +34244 694.54 +34289 185.70 +34338 411.46 +34466 1736.55 +34536 2259.62 +34593 346.48 +34694 592.38 +34843 461.27 +34854 448.63 +35038 1972.67 +35124 415.20 +35281 538.84 +35709 1008.27 +35752 1067.71 +35804 1004.19 +35817 418.39 +36060 659.72 +36074 509.22 +36098 2175.73 +36192 1679.14 +36277 554.20 +36534 982.64 +36557 437.46 +36787 2030.37 +36788 357.97 +36867 649.86 +36871 551.07 +36971 473.15 +36997 953.02 +37057 832.78 +37411 447.31 +37683 1675.77 +37745 689.08 +37752 871.12 +37838 1238.05 +37940 257.25 +38014 1047.08 +38059 374.46 +38075 1052.41 +38095 935.02 +38354 310.51 +38370 2677.66 +38371 1890.06 +38482 1035.92 +38579 1957.82 +38605 652.30 +38721 1593.98 +38784 739.06 +38828 832.91 +38877 669.68 +38883 1743.88 +39003 400.20 +39101 481.37 +39145 633.33 +39231 576.23 +39237 895.30 +39275 622.01 +39303 825.38 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q16.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q16.sql.out new file mode 100644 index 0000000000000..6de87c375ca54 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q16.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +280 1300478.25 -285761.26 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q17.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q17.sql.out new file mode 100644 index 0000000000000..a3facf3ecb9ec --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q17.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q18.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q18.sql.out new file mode 100644 index 0000000000000..cac227c379b81 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q18.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL 50.804533 99.251888 256.393484 49.082603 -338.277102 1957.826868 3.274912 +AAAAAAAAAACDAAAA NULL NULL NULL 8.000000 204.490000 0.000000 47.030000 -299.760000 1948.000000 2.000000 +AAAAAAAAAADCAAAA NULL NULL NULL 46.000000 178.690000 0.000000 146.520000 2502.860000 1962.000000 0.000000 +AAAAAAAAAAGBAAAA NULL NULL NULL 86.000000 1.520000 40.290000 0.660000 -97.910000 1945.000000 2.000000 +AAAAAAAAAAGCAAAA NULL NULL NULL 84.000000 41.920000 0.000000 8.380000 -640.080000 1945.000000 2.000000 +AAAAAAAAAAGEAAAA NULL NULL NULL 99.000000 107.550000 0.000000 65.600000 738.540000 1973.000000 3.000000 +AAAAAAAAAAJBAAAA NULL NULL NULL 97.000000 48.580000 0.000000 44.690000 534.470000 1984.000000 1.000000 +AAAAAAAAAAKAAAAA NULL NULL NULL 36.000000 164.023333 0.000000 57.763333 -1073.803333 1962.666667 2.333333 +AAAAAAAAAAOCAAAA NULL NULL NULL 82.000000 148.640000 0.000000 107.020000 4483.760000 1948.000000 6.000000 +AAAAAAAAAAODAAAA NULL NULL NULL 100.000000 80.700000 0.000000 64.560000 1505.000000 1981.000000 4.000000 +AAAAAAAAABADAAAA NULL NULL NULL 55.000000 134.120000 3189.310000 63.030000 -2198.210000 1945.000000 2.000000 +AAAAAAAAABCEAAAA NULL NULL NULL 25.000000 150.360000 85.690000 9.020000 -1702.940000 1948.000000 6.000000 +AAAAAAAAABDEAAAA NULL NULL NULL 24.000000 99.490000 0.000000 94.510000 562.560000 1958.000000 0.000000 +AAAAAAAAABFBAAAA NULL NULL NULL 2.000000 60.810000 0.000000 45.600000 -26.880000 1931.000000 6.000000 +AAAAAAAAABFCAAAA NULL NULL NULL 65.000000 45.000000 0.000000 23.310000 -1071.285000 1937.500000 3.000000 +AAAAAAAAABGBAAAA NULL NULL NULL 65.000000 20.450000 0.000000 0.810000 -401.050000 1983.000000 2.000000 +AAAAAAAAABJBAAAA NULL NULL NULL 42.000000 69.930000 0.000000 49.650000 260.820000 1943.000000 2.000000 +AAAAAAAAABNCAAAA NULL NULL NULL 81.000000 140.170000 0.000000 86.900000 1733.400000 1953.000000 4.000000 +AAAAAAAAACCBAAAA NULL NULL NULL 50.000000 99.750000 0.000000 75.810000 1109.000000 1945.000000 1.000000 +AAAAAAAAACCDAAAA NULL NULL NULL 71.000000 92.520000 0.000000 25.900000 -4026.410000 1928.000000 0.000000 +AAAAAAAAACDDAAAA NULL NULL NULL 29.000000 147.490000 0.000000 35.390000 -741.240000 1933.000000 5.000000 +AAAAAAAAACIBAAAA NULL NULL NULL 4.000000 38.470000 0.000000 35.770000 41.160000 1952.000000 2.000000 +AAAAAAAAACPDAAAA NULL NULL NULL 91.000000 86.510000 0.000000 20.760000 -1989.260000 1981.000000 3.000000 +AAAAAAAAADACAAAA NULL NULL NULL 53.000000 227.940000 0.000000 113.970000 1089.150000 1954.000000 6.000000 +AAAAAAAAADBDAAAA NULL NULL NULL 70.000000 35.050000 0.000000 29.440000 970.200000 1990.000000 2.000000 +AAAAAAAAADEEAAAA NULL NULL NULL 66.000000 85.760000 2913.510000 44.590000 -3553.050000 1926.000000 3.000000 +AAAAAAAAADJBAAAA NULL NULL NULL 28.000000 90.830000 0.000000 15.155000 -466.175000 1986.000000 1.000000 +AAAAAAAAADPBAAAA NULL NULL NULL 55.000000 47.460000 0.000000 18.500000 43.450000 1989.000000 2.000000 +AAAAAAAAAEABAAAA NULL NULL NULL 62.000000 89.450000 0.000000 61.720000 225.060000 1958.000000 0.000000 +AAAAAAAAAEBAAAAA NULL NULL NULL 93.000000 175.980000 0.000000 156.620000 6503.490000 1952.000000 6.000000 +AAAAAAAAAECCAAAA NULL NULL NULL 97.000000 179.380000 0.000000 127.350000 4183.610000 1952.000000 1.000000 +AAAAAAAAAEDDAAAA NULL NULL NULL 81.000000 104.250000 0.000000 53.160000 -92.340000 1982.000000 0.000000 +AAAAAAAAAEGDAAAA NULL NULL NULL 6.000000 170.770000 61.470000 102.460000 103.890000 1985.000000 0.000000 +AAAAAAAAAEGEAAAA NULL NULL NULL 97.000000 34.300000 0.000000 24.350000 392.850000 1976.000000 6.000000 +AAAAAAAAAEJDAAAA NULL NULL NULL 74.000000 168.330000 0.000000 143.080000 3926.440000 1930.000000 1.000000 +AAAAAAAAAENAAAAA NULL NULL NULL 15.000000 73.420000 0.000000 23.490000 -108.450000 1963.000000 2.000000 +AAAAAAAAAEPAAAAA NULL NULL NULL 34.000000 4.380000 0.000000 3.460000 67.320000 1952.000000 1.000000 +AAAAAAAAAEPDAAAA NULL NULL NULL 54.000000 99.220000 1013.550000 43.650000 -3909.570000 1958.000000 0.000000 +AAAAAAAAAFGAAAAA NULL NULL NULL 74.000000 90.960000 0.000000 0.900000 -4919.520000 1964.000000 6.000000 +AAAAAAAAAFGCAAAA NULL NULL NULL 79.000000 51.830000 0.000000 36.280000 787.630000 1975.000000 5.000000 +AAAAAAAAAFGDAAAA NULL NULL NULL 80.000000 38.080000 0.000000 0.000000 -1116.000000 1926.000000 0.000000 +AAAAAAAAAFHBAAAA NULL NULL NULL 57.000000 55.990000 0.000000 31.910000 320.340000 1970.000000 6.000000 +AAAAAAAAAFLBAAAA NULL NULL NULL 92.000000 44.330000 0.000000 41.220000 836.280000 1943.000000 3.000000 +AAAAAAAAAGBEAAAA NULL NULL NULL 6.000000 260.570000 0.000000 93.800000 14.220000 1970.000000 2.000000 +AAAAAAAAAGCAAAAA NULL NULL NULL 54.000000 231.920000 0.000000 48.700000 -2187.000000 1969.000000 5.000000 +AAAAAAAAAGCCAAAA NULL NULL NULL 80.000000 183.680000 0.000000 99.180000 475.200000 1946.000000 0.000000 +AAAAAAAAAGCDAAAA NULL NULL NULL 69.000000 143.220000 2478.280000 54.420000 -4676.620000 1970.000000 6.000000 +AAAAAAAAAGEDAAAA NULL NULL NULL 93.000000 60.830000 0.000000 52.310000 2769.540000 1942.000000 6.000000 +AAAAAAAAAGIDAAAA NULL NULL NULL 28.000000 75.550000 0.000000 9.060000 -1013.040000 1942.000000 6.000000 +AAAAAAAAAGJBAAAA NULL NULL NULL 36.000000 7.170000 0.000000 5.230000 65.880000 1984.000000 6.000000 +AAAAAAAAAGKAAAAA NULL NULL NULL 33.000000 95.840000 0.000000 4.790000 -1480.710000 1948.000000 6.000000 +AAAAAAAAAGLAAAAA NULL NULL NULL 85.000000 143.410000 570.460000 74.570000 276.990000 1948.000000 6.000000 +AAAAAAAAAGLCAAAA NULL NULL NULL 65.000000 13.730000 0.000000 3.980000 -176.800000 1976.000000 6.000000 +AAAAAAAAAHADAAAA NULL NULL NULL 52.000000 33.050000 0.000000 10.900000 -1054.560000 1936.000000 1.000000 +AAAAAAAAAHBCAAAA NULL NULL NULL 20.000000 187.740000 0.000000 31.910000 -994.400000 1947.000000 5.000000 +AAAAAAAAAHBDAAAA NULL NULL NULL 61.000000 83.490000 0.000000 59.270000 1201.700000 1945.000000 2.000000 +AAAAAAAAAHCBAAAA NULL NULL NULL 2.000000 136.960000 0.000000 0.000000 -95.780000 1970.000000 5.000000 +AAAAAAAAAHDEAAAA NULL NULL NULL 15.000000 31.950000 232.130000 16.290000 -164.630000 1934.000000 2.000000 +AAAAAAAAAHGAAAAA NULL NULL NULL 99.000000 100.770000 12.870000 1.000000 -9600.030000 1932.000000 4.000000 +AAAAAAAAAHHAAAAA NULL NULL NULL 18.000000 109.980000 0.000000 24.190000 -634.680000 1933.000000 2.000000 +AAAAAAAAAHIBAAAA NULL NULL NULL 86.000000 20.920000 0.000000 5.230000 -1332.140000 1990.000000 1.000000 +AAAAAAAAAHJDAAAA NULL NULL NULL 53.000000 88.370000 0.000000 37.990000 -1308.570000 1948.000000 2.000000 +AAAAAAAAAHMDAAAA NULL NULL NULL 68.000000 200.900000 546.440000 20.090000 -4993.640000 NULL 6.000000 +AAAAAAAAAHPDAAAA NULL NULL NULL 6.000000 73.210000 60.840000 13.170000 -276.660000 1936.000000 6.000000 +AAAAAAAAAIACAAAA NULL NULL NULL 1.000000 17.830000 0.000000 10.160000 1.940000 1935.000000 1.000000 +AAAAAAAAAIADAAAA NULL NULL NULL 100.000000 22.020000 0.000000 15.630000 -131.000000 1982.000000 0.000000 +AAAAAAAAAIEBAAAA NULL NULL NULL 4.000000 51.860000 0.000000 42.520000 -8.760000 1967.000000 4.000000 +AAAAAAAAAIEEAAAA NULL NULL NULL 64.000000 18.810000 0.000000 15.610000 468.480000 1930.000000 1.000000 +AAAAAAAAAINBAAAA NULL NULL NULL 6.000000 102.610000 2.950000 24.620000 -110.710000 1930.000000 2.000000 +AAAAAAAAAIODAAAA NULL NULL NULL 81.000000 42.490000 0.000000 12.740000 -339.390000 1990.000000 1.000000 +AAAAAAAAAJBEAAAA NULL NULL NULL 95.000000 118.220000 0.000000 15.360000 -3863.650000 1962.000000 0.000000 +AAAAAAAAAJDCAAAA NULL NULL NULL 85.000000 190.480000 0.000000 125.710000 4732.800000 1989.000000 2.000000 +AAAAAAAAAJEAAAAA NULL NULL NULL 36.000000 183.430000 0.000000 0.000000 -3318.480000 1942.000000 6.000000 +AAAAAAAAAJGCAAAA NULL NULL NULL 43.500000 39.415000 619.885000 28.940000 -340.325000 1960.000000 5.000000 +AAAAAAAAAJIAAAAA NULL NULL NULL 69.000000 118.330000 0.000000 61.530000 1117.110000 1970.000000 2.000000 +AAAAAAAAAJICAAAA NULL NULL NULL 39.333333 102.063333 0.000000 24.780000 -848.446667 1962.666667 2.333333 +AAAAAAAAAJIDAAAA NULL NULL NULL 75.000000 222.870000 6498.720000 180.520000 -42.720000 1935.000000 5.000000 +AAAAAAAAAJJBAAAA NULL NULL NULL 42.000000 103.240000 0.000000 43.360000 -149.940000 1932.000000 4.000000 +AAAAAAAAAJNAAAAA NULL NULL NULL 93.000000 14.470000 0.000000 13.740000 136.710000 1954.000000 0.000000 +AAAAAAAAAJOCAAAA NULL NULL NULL 24.000000 88.660000 0.000000 2.650000 -1000.320000 1927.000000 2.000000 +AAAAAAAAAJPBAAAA NULL NULL NULL 16.000000 159.510000 0.000000 121.220000 1085.920000 1983.000000 4.000000 +AAAAAAAAAKAEAAAA NULL NULL NULL 5.000000 178.480000 0.000000 57.110000 -160.650000 1990.000000 2.000000 +AAAAAAAAAKFCAAAA NULL NULL NULL 84.000000 284.260000 0.000000 247.300000 12760.440000 1984.000000 1.000000 +AAAAAAAAAKGDAAAA NULL NULL NULL 26.000000 132.060000 0.000000 64.700000 -291.200000 1930.000000 2.000000 +AAAAAAAAAKHCAAAA NULL NULL NULL 99.000000 97.000000 0.000000 19.400000 -3327.390000 1935.000000 1.000000 +AAAAAAAAALBEAAAA NULL NULL NULL 61.000000 7.760000 0.000000 2.630000 -125.050000 1981.000000 0.000000 +AAAAAAAAALFBAAAA NULL NULL NULL 58.000000 92.090000 0.000000 35.910000 -97.440000 1970.000000 3.000000 +AAAAAAAAALGCAAAA NULL NULL NULL 77.000000 36.240000 0.000000 18.840000 -66.220000 1983.000000 4.000000 +AAAAAAAAALLBAAAA NULL NULL NULL 82.000000 53.000000 0.000000 8.480000 -840.500000 1981.000000 4.000000 +AAAAAAAAALMDAAAA NULL NULL NULL 86.000000 32.020000 802.660000 17.610000 -1020.240000 1963.000000 6.000000 +AAAAAAAAALOAAAAA NULL NULL NULL 78.000000 91.520000 0.000000 73.210000 44.460000 1964.000000 6.000000 +AAAAAAAAAMBAAAAA NULL NULL NULL 79.000000 93.970000 0.000000 93.970000 4924.070000 1948.000000 2.000000 +AAAAAAAAAMDBAAAA NULL NULL NULL 15.000000 79.140000 0.000000 18.990000 -421.800000 1969.000000 2.000000 +AAAAAAAAAMEAAAAA NULL NULL NULL 10.000000 164.570000 0.000000 80.630000 -123.500000 1934.000000 5.000000 +AAAAAAAAAMFDAAAA NULL NULL NULL 2.000000 41.000000 0.000000 17.220000 -4.800000 1948.000000 6.000000 +AAAAAAAAAMGBAAAA NULL NULL NULL 66.000000 12.410000 0.000000 3.590000 -140.580000 1990.000000 0.000000 +AAAAAAAAAMLDAAAA NULL NULL NULL 70.000000 147.430000 297.020000 8.840000 -3444.920000 1943.000000 4.000000 +AAAAAAAAAMNAAAAA NULL NULL NULL 41.000000 53.225000 0.000000 37.950000 121.915000 1960.000000 5.000000 +AAAAAAAAANABAAAA NULL NULL NULL 94.000000 8.250000 30.830000 0.410000 -293.090000 1986.000000 6.000000 +AAAAAAAAANAEAAAA NULL NULL NULL 5.000000 47.390000 0.000000 9.470000 -151.800000 1931.000000 2.000000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q19.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q19.sql.out new file mode 100644 index 0000000000000..29347b2d590bc --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q19.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +10012006 importoamalgamalg #x 221 oughtableable 56586.14 +6015008 scholarbrand #x 66 callycally 46201.70 +4001001 amalgedu pack #x 950 barantin st 43315.50 +3002002 importoexporti #x 752 ableantiation 42749.12 +2003001 exportiimporto #x 586 callyeinganti 41753.03 +6009007 maxicorp #x 120 barableought 41183.43 +8010005 univmaxi #x 520 barableanti 40133.14 +7005005 scholarbrand #x 187 ationeingought 37940.95 +10004015 edu packunivamalg #x 439 n stpriese 36037.04 +9015005 scholarunivamalg #x 297 ationn stable 34881.18 +5003001 exportischolar #x 227 ationableable 34528.86 +2002001 importoimporto #x 536 callyprianti 34412.64 +7007003 brandbrand #x 759 n stantiation 33533.84 +7016003 corpnameless #x 271 oughtationable 32722.68 +10003006 exportiunivamalg #x 105 antibarought 31509.32 +3003001 exportiexporti #x 178 eingationought 30944.05 +2003001 exportiimporto #x 269 n stcallyable 30743.96 +7012007 importonameless #x 578 eingationanti 30629.71 +9010011 univunivamalg #x 27 ationable 30165.17 +1001001 amalgamalg #x 390 barn stpri 29510.42 +1004002 edu packamalg #x 86 callyeing 28798.78 +8016010 corpmaxi #x 745 antieseation 28399.44 +3003002 exportiexporti #x 52 ableanti 28330.70 +7002007 importobrand #x 261 oughtcallyable 28076.07 +2002001 importoimporto #x 364 esecallypri 27831.19 +5001002 amalgscholar #x 743 prieseation 27622.14 +8006009 corpnameless #x 148 eingeseought 26685.21 +2002001 importoimporto #x 68 eingcally 26391.94 +8004005 edu packnameless #x 192 ablen stought 26231.73 +1004001 edu packamalg #x 583 prieinganti 26107.88 +1001001 amalgamalg #x 282 ableeingable 26016.70 +8006009 corpnameless #x 319 n stoughtpri 25529.26 +6016003 corpbrand #x 110 baroughtought 25233.47 +9007003 brandmaxi #x 34 esepri 25164.92 +2004001 edu packimporto #x 5 anti 25083.59 +5002001 importoscholar #x 582 ableeinganti 24752.75 +3001001 amalgexporti #x 296 callyn stable 24732.94 +7007007 brandbrand #x 529 n stableanti 24268.11 +8010009 univmaxi #x 777 ationationation 24160.84 +8007002 brandnameless #x 192 ablen stought 23590.40 +8014006 edu packmaxi #x 4 ese 23430.31 +6005005 scholarcorp #x 129 n stableought 23382.47 +8015001 scholarmaxi #x 78 eingation 23235.50 +6004007 edu packcorp #x 158 eingantiought 23188.37 +9016003 corpunivamalg #x 304 esebarpri 23156.77 +10010013 univamalgamalg #x 591 oughtn stanti 23127.55 +2004001 edu packimporto #x 563 pricallyanti 22985.88 +2003002 exportiimporto #x 490 barn stese 22045.36 +1003002 exportiamalg #x 376 callyationpri 21890.84 +1004001 edu packamalg #x 39 n stpri 21878.31 +1001001 amalgamalg #x 760 barcallyation 21856.16 +9014002 edu packunivamalg #x 78 eingation 21725.66 +1004001 edu packamalg #x 366 callycallypri 21166.78 +1004001 edu packamalg #x 513 prioughtanti 21125.73 +1002001 importoamalg #x 942 ableesen st 21119.54 +5002002 importoscholar #x 102 ablebarought 21049.49 +5002001 importoscholar #x 75 antiation 20510.26 +8002004 importonameless #x 2 able 20401.46 +8011002 amalgmaxi #x 29 n stable 20270.33 +5001001 amalgscholar #x 127 ationableought 19976.77 +1001001 amalgamalg #x 522 ableableanti 19773.14 +7003005 exportibrand #x 45 antiese 19662.73 +1002001 importoamalg #x 361 oughtcallypri 19619.15 +7007002 brandbrand #x 410 baroughtese 18996.04 +NULL brandcorp #x NULL NULL 18842.73 +9008008 namelessmaxi #x 607 ationbarcally 18650.79 +8015010 scholarmaxi #x 410 baroughtese 18465.35 +10008005 namelessunivamalg #x 349 n stesepri 18229.28 +9001008 amalgmaxi #x 840 bareseeing 17824.97 +2004002 edu packimporto #x 314 eseoughtpri 17523.17 +4003001 exportiedu pack #x 873 priationeing 17462.90 +9007005 brandmaxi #x 546 callyeseanti 17405.98 +10013015 exportiamalgamalg #x 427 ationableese 17058.19 +3001001 amalgexporti #x 139 n stpriought 16994.07 +8015002 scholarmaxi #x 15 antiought 16963.84 +4003002 exportiedu pack #x 664 esecallycally 16936.98 +6005001 scholarcorp #x 404 esebarese 16500.58 +2003002 exportiimporto #x 333 pripripri 16133.68 +9006003 corpmaxi #x 494 esen stese 16104.95 +6009007 maxicorp #x 468 eingcallyese 16083.46 +4002002 importoedu pack #x 408 eingbarese 15980.04 +8012009 importomaxi #x 351 oughtantipri 15946.21 +4001002 amalgedu pack #x 102 ablebarought 15911.13 +6011007 amalgbrand #x 822 ableableeing 15556.11 +4004002 edu packedu pack #x 51 oughtanti 15506.23 +4003002 exportiedu pack #x 12 ableought 15452.21 +10010010 univamalgamalg #x 721 oughtableation 15210.55 +3001001 amalgexporti #x 574 eseationanti 14927.08 +3002002 importoexporti #x 382 ableeingpri 14658.43 +3001001 amalgexporti #x 142 ableeseought 14633.69 +10002014 importounivamalg #x 153 priantiought 14599.02 +1004001 edu packamalg #x 148 eingeseought 14327.50 +5003002 exportischolar #x 164 esecallyought 14289.22 +7008009 namelessbrand #x 532 ableprianti 14197.70 +4003001 exportiedu pack #x 258 eingantiable 14148.81 +7007001 brandbrand #x 281 oughteingable 13920.21 +2002001 importoimporto #x 258 eingantiable 13819.06 +8007007 brandnameless #x 423 priableese 13589.86 +4004001 edu packedu pack #x 456 callyantiese 13461.44 +8013009 exportimaxi #x 599 n stn stanti 13459.83 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q2.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q2.sql.out new file mode 100644 index 0000000000000..44e1f7bfef7fa --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q2.sql.out @@ -0,0 +1,2518 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5270 3.18 1.63 2.25 1.64 3.41 3.62 3.72 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5271 1.00 1.15 1.23 0.82 1.06 0.85 0.95 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5272 1.22 0.86 1.29 0.95 0.98 1.00 0.84 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5273 1.20 0.99 1.15 0.86 0.91 1.19 0.90 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5274 0.97 0.95 1.08 1.19 0.97 0.89 0.96 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5275 0.99 0.89 1.42 1.00 0.84 1.11 0.98 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5276 0.97 1.06 1.16 0.86 0.98 1.25 1.13 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5277 0.91 0.91 1.02 0.95 1.00 1.05 0.90 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5278 0.96 0.94 1.29 1.04 0.95 2.06 1.46 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5279 0.87 0.86 0.88 0.99 0.38 0.83 0.77 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5280 0.96 0.89 0.73 1.09 1.09 0.91 1.21 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5281 0.99 1.03 1.09 0.86 0.92 1.13 1.01 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5282 0.97 0.89 1.08 1.10 1.15 0.86 1.33 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5283 1.27 0.85 1.07 0.88 1.23 0.93 1.27 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5284 1.06 0.92 0.95 1.04 0.95 1.29 0.89 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5285 1.10 1.20 0.97 1.14 1.11 1.12 0.95 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5286 1.09 1.17 1.17 0.86 1.12 1.39 1.00 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5287 0.94 1.00 0.99 0.76 1.10 0.90 0.96 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5288 0.95 1.07 1.40 1.05 0.94 0.94 1.01 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5289 0.99 1.20 0.90 0.95 1.01 0.82 0.76 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5290 1.05 1.10 1.04 1.22 0.97 0.98 0.98 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5291 0.78 1.28 0.90 0.99 0.97 1.09 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5292 1.13 0.79 1.17 1.07 0.88 0.85 0.91 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5293 1.29 0.96 0.89 0.87 0.97 0.89 1.06 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5294 1.18 1.05 1.25 0.98 0.88 1.02 0.69 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5295 0.87 1.08 1.03 1.01 0.88 1.27 1.12 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5296 0.74 1.11 0.96 1.12 1.00 1.25 1.02 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5297 1.04 0.97 0.98 0.79 0.90 0.90 1.03 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5298 0.80 0.78 0.94 1.04 1.26 0.91 0.90 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5299 0.87 1.00 1.14 1.13 1.07 1.20 1.01 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5300 0.53 0.41 0.98 0.67 0.80 0.56 0.41 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5301 0.95 1.05 0.38 0.50 0.76 1.10 1.14 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5302 1.03 0.80 0.85 1.12 1.00 0.96 1.01 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5303 0.77 1.23 0.94 1.09 0.98 0.96 0.98 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5304 0.96 0.82 1.13 1.04 1.07 1.24 1.00 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5305 1.01 0.96 0.91 0.91 0.90 1.15 1.09 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5306 0.83 0.99 0.97 1.04 0.90 1.00 1.03 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5307 0.97 0.99 0.89 0.88 1.07 1.02 0.94 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5308 0.93 0.95 1.10 0.97 1.10 0.89 1.04 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5309 1.20 0.90 1.05 1.11 0.96 1.05 1.06 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5310 0.97 1.10 1.12 1.06 0.96 1.03 0.87 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5311 1.09 0.88 1.06 1.01 0.91 1.10 1.03 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5312 1.06 0.93 0.78 0.99 1.03 0.87 1.07 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5313 0.62 0.78 1.13 1.03 1.13 1.04 0.85 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5314 0.98 1.16 0.71 0.57 0.70 1.07 1.06 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5315 1.10 1.09 0.99 1.02 1.11 1.00 1.02 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5316 1.01 1.07 1.09 1.01 1.08 1.03 0.87 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5317 0.96 1.04 0.94 0.99 1.09 0.95 0.88 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5318 0.94 1.00 1.18 0.94 0.87 1.02 1.11 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5319 0.97 0.96 0.96 1.12 1.00 0.93 0.88 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5320 1.06 0.94 0.97 1.14 0.96 0.90 1.04 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5321 0.96 1.01 0.96 1.03 0.97 0.95 0.89 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 +5322 4.54 4.72 0.99 1.74 1.43 4.36 5.11 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q20.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q20.sql.out new file mode 100644 index 0000000000000..3c2757f0645f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q20.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL Books NULL NULL 9010.49 72.07596281370536693 +NULL Books NULL 6.35 1491.96 11.93436244638591899 +Precisely elderly bodies Books arts 1.40 4094.31 1.52075020571659240 +Close, precise teeth should go for a qualities. Political groups shall not become just important occasions. Trials mean ne Books arts 2.53 332.38 0.12345595555199313 +Abilities could affect cruel parts. Predominantly other events telephone strong signs. Accurate mate Books arts 25.69 2626.56 0.97558359291967949 +Average parents require also times. Children would not describe lightly purposes; large miles love now correct relations. Usual, german goals proceed literary, wooden visitors. Initial councils wil Books arts 1.24 12327.20 4.57869383019594946 +Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 343.80 0.12769768794384511 +Small objects stop etc mediterranean patterns; liberal, free initiatives would not leave less clear british attitudes; good, blue relationships find softly very Books arts 58.41 886.92 0.32942883476194038 +Newly national rights head curiously all electrical cells. Chinese, long values might not pull bad lines. High fun clothes ough Books arts 3.28 2219.85 0.82451923380495801 +Quick, easy studies must make always necessary systems. Upper, new persons should buy much physical technologies. English sciences hear solicitors. S Books arts 0.99 2050.16 0.76149125047979491 +Early, short v Books arts 75.57 5429.86 2.01681375177070042 +Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 13539.35 5.02892289488801418 +Scientific, difficult polls would not achieve. Countries reach of course. Bad, new churches realize most english Books arts 3.98 143.88 0.05344137097545211 +United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 12297.15 4.56753235398096242 +Naturally new years put serious, negative vehicles. Fin Books arts 3.34 4587.47 1.70392470189572752 +Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Books arts 4.03 5359.20 1.99056849688381241 +Much new waters Books arts 1.85 6718.63 2.49550179508480530 +Hard different differences would not paint even. Together suitable schemes marry directly only open women. Social ca Books arts 2.65 3208.60 1.19177080144450674 +Tall, following actions keep widely willing, secondary groups. Heads could afford however; agricultural, square pri Books arts 9.99 4780.52 1.77562929368618505 +Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 5622.46 2.08835119999055082 +Powerful walls will find; there scottish decades must not Books arts 4.16 7914.41 2.93965054810833964 +Too executive doors progress mainly seemingly possible parts; hundreds stay virtually simple workers. Sola Books arts 34.32 3029.48 1.12524023173973205 +Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 1450.99 0.53894144336718969 +Aside legitimate decisions may not stand probably sexual g Books arts 3.88 9619.83 3.57309496629679899 +Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 13616.57 5.05760473160419719 +Unlikely states take later in general extra inf Books arts 0.32 11879.56 4.41242683475911751 +Sometimes careful things state probably so Books arts 5.08 25457.85 9.45581321995700176 +Circumstances would not use. Principles seem writers. Times go from a hands. Members find grounds. Central, only teachers pursue properly into a p Books arts 5.95 2567.54 0.95366178505916251 +Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 1811.85 0.67297572978782944 +Systems cannot await regions. Home appropr Books arts 7.30 1730.16 0.64263360027028230 +Extra, primitive weeks look obviou Books arts 1.18 22.77 0.00845746467272063 +More than key reasons should remain. Words used to offer slowly british Books arts 0.28 10311.18 3.82988320527288194 +Children may turn also above, historical aspects. Surveys migh Books arts 7.22 11872.32 4.40973768042312729 +Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 8348.99 3.10106666569599586 +Useful observers start often white colleagues; simple pro Books arts 3.47 7565.51 2.81005856636428042 +Members should say earnings. Detailed departments would not move just at the hopes. Figures can take. Actually open houses want. Good teachers combine the Books arts 3.09 4363.97 1.62091006182752106 +Major, senior words afford economic libraries; successful seconds need outside. Clinical, new ideas put now red c Books arts 5.87 9661.08 3.58841646026911898 +Likely states feel astonishingly working roads. Parents put so somewhere able policies. Others may rely shortly instead interesting bodies; bri Books arts 7.50 132.66 0.04927392461498107 +Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 20114.53 7.47114303396483641 +Accurate years want then other organisations. Simple lines mean as well so red results. Orthodox, central scales will not in Books arts 7.69 2153.04 0.79970398502215321 +Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 231.71 0.08606408165639427 +Visitors could not allow glad wages. Communist, real figures used to apply factors. Aggressive, optimistic days must mean about trees. Detailed courts consider really large pro Books arts 9.08 24425.09 9.07221501111207600 +Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 4380.23 1.62694951617879192 +New, reluctant associations see more different, physical symptoms; useful pounds ought to give. Subjects Books business 9.02 3044.02 1.58609001939612781 +Imports involve most now indian women. Developments announce intimately in a copies. Projects Books business 3.26 472.29 0.24608723177265498 +Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 30227.05 15.74983814849696292 +Whole, important problems make. Indeed industrial members go skills. Soft Books business 3.22 137.92 0.07186336997625310 +Other, black houses flow. New soldiers put only eastern hours. Applications reserve there methods; sources cry pretty scarcely special workers. Never british opportunities Books business 8.20 736.96 0.38399383075478162 +Rows could not Books business 1.65 1290.88 0.67261446516056841 +Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 1065.30 0.55507575431918810 +White members see highly on a negotiations. Evident, passive colours can refer familiar, ugly factors; away small examinations shall prove Books business 17.97 1446.00 0.75343991433919646 +Manufacturing, ready concerns see already then new pupils. Both stable types used to manage otherw Books business 1.18 2635.71 1.37333963805184198 +Small, capable centres Books business 2.98 5029.45 2.62060053746422658 +Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 621.26 0.32370821658531756 +Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 3950.22 2.05826654109334761 +Always other hours used to use. Women should jump then. Civil samples take therefore other offices. Concrete, major demands Books business 1.42 2013.79 1.04928752772968910 +Visual fragments Books business 6.77 930.13 0.48464527491308216 +Classic issues will draw as european, engl Books business 75.64 556.83 0.29013689315456070 +Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 1898.13 0.98902275560488173 +Confident, video-tape Books business 3.17 1131.00 0.58930881266779474 +Of course fundamental children will not deal still including a suppliers. More crucial powers will not keep enough. As good comments used to devote even convenient electric problems. Publi Books business 8.85 414.75 0.21610595053401226 +Departments could seek now for a commu Books business 5.93 9895.85 5.15624369039663714 +Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 13809.44 7.19542412909562460 +Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 9267.25 4.82871095861681771 +So white countries could secure more angry items. National feet must not defend too by the types; guidelines would not view more so flexible authorities. Critics will handle closely lig Books business 2.50 2542.50 1.32477246349059959 +Simple changes ought to vote almost sudden techniques. Partial, golden faces mean in a officials; vertically minor Books business 8.73 22710.22 11.83318548507904997 +Christian lines stand once deep formal aspirations. National, fine islands play together with a patterns. New journals lose etc positive armie Books business 4.89 11560.78 6.02375732565303988 +Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 9065.09 4.72337526492192700 +Lips will n Books business 8.48 541.62 0.28221170567385587 +White fees might combine reports. Tr Books business 2.09 37.60 0.01959152197728478 +Asleep children invite more. Wealthy forms could expect as. Indeed statistical examinations could la Books business 3.71 2082.24 1.08495347664844290 +Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 13412.42 6.98855641485568838 +Local, bloody names Books business 4.40 1997.44 1.04076834197626873 +Large, larg Books business 3.50 12097.82 6.30358261721370521 +Only, gothic Books business 1.68 5708.95 2.97465477106967886 +Low, large clouds will not visit for example as the notions. Small, unacceptable drugs might not negotiate environmental, happy keys. Books business 3.11 3020.85 1.57401726502874248 +Silver, critical operations could help howev Books business 5.56 2286.24 1.19124790439754116 +Terrible, psychiatric bones will destroy also used studies; solely usual windows should not make shares. Advances continue sufficiently. As key days might not use far artists. Offici Books business 5.83 6672.40 3.47666146918178041 +Too white addresses end by the talks. Hands get only companies. Statements know. Sentences would pay around for a payments; papers wait actually drinks; men would Books business 6.06 7609.35 3.96486031270882752 +New, big arguments may not win since by a tenant Books computers 1.00 904.16 0.32327741862037314 +Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 18752.88 6.70498876094676063 +Cheap, desirable members take immediate, estimated debts; months must track typica Books computers 3.26 10027.86 3.58540600677589698 +Expert, scottish terms will ask quiet demands; poor bits attempt northern, dangerous si Books computers 2.66 7330.68 2.62104418148557444 +Gradually serious visitors bear no doubt technical hearts. Critics continue earlier soviet, standard minute Books computers 6.40 1711.84 0.61205894564136830 +Clear, general goods must know never women. Communications meet about. Other rewards spot wide in a skills. Relative, empty drawings facilitate too rooms. Still asian police end speedily comp Books computers 7.64 1292.04 0.46196177220211789 +At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 3033.10 1.08446816760026298 +Environmental, new women pay again fingers. Different, uncomfortable records miss far russian, dependent members. Enough double men will go here immediatel Books computers 89.89 8553.39 3.05821739476786568 +Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 3411.40 1.21972724504682903 +Magnetic Books computers 57.19 3569.09 1.27610843437421206 +Ga Books computers 5.53 2687.70 0.96097230360331899 +S Books computers 65.78 1613.04 0.57673355084432699 +Simple year Books computers 3.01 1262.79 0.45150359611088856 +Agricultural players shall smoke. So full reasons undertake Books computers 0.70 4408.27 1.57615261257037727 +Then basic years can encourage later traditions. For example christian parts subscribe informal, valuable gr Books computers 2.75 844.19 0.30183547604973987 +Boxes batt Books computers 0.83 15300.82 5.47072375727191844 +Separate, dead buildings think possibly english, net policies. Big divisions can use almost Books computers 9.46 12403.71 4.43487806374503246 +Artists make times. Rather ready functions must pre Books computers 5.71 1533.00 0.54811569052494252 +Advantages emerge moves; special, expected operations pass etc natural preferences; very posit Books computers 0.15 5241.45 1.87405152387603389 +Since other birds shall blame sudden Books computers 6.74 2098.16 0.75018552983158082 +Legs throw then. Old-fashioned develo Books computers 2.66 163.26 0.05837271209073850 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q21.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q21.sql.out new file mode 100644 index 0000000000000..9d7ddb1678a57 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q21.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Just good amou AAAAAAAAAAEAAAAA 2473 2677 +Just good amou AAAAAAAAAAPBAAAA 2415 2071 +Just good amou AAAAAAAAACACAAAA 2792 2465 +Just good amou AAAAAAAAACGCAAAA 1396 2084 +Just good amou AAAAAAAAACKCAAAA 1974 1585 +Just good amou AAAAAAAAACPCAAAA 2131 1690 +Just good amou AAAAAAAAADACAAAA 2432 2093 +Just good amou AAAAAAAAADBBAAAA 1747 2529 +Just good amou AAAAAAAAADNAAAAA 2763 1862 +Just good amou AAAAAAAAAELBAAAA 2984 2371 +Just good amou AAAAAAAAAFFDAAAA 2858 2415 +Just good amou AAAAAAAAAFJDAAAA 2479 2176 +Just good amou AAAAAAAAAFLDAAAA 2892 2303 +Just good amou AAAAAAAAAGLDAAAA 2083 1892 +Just good amou AAAAAAAAAGPCAAAA 1596 1191 +Just good amou AAAAAAAAAHBAAAAA 2398 2956 +Just good amou AAAAAAAAAHPBAAAA 1775 2655 +Just good amou AAAAAAAAAINCAAAA 2296 2458 +Just good amou AAAAAAAAAJIDAAAA 2528 2552 +Just good amou AAAAAAAAAKABAAAA 1881 1465 +Just good amou AAAAAAAAALFEAAAA 2952 2933 +Just good amou AAAAAAAAALIDAAAA 1918 2438 +Just good amou AAAAAAAAALOAAAAA 1904 1396 +Just good amou AAAAAAAAALOBAAAA 1340 1570 +Just good amou AAAAAAAAALPAAAAA 2299 1692 +Just good amou AAAAAAAAANLCAAAA 3000 3113 +Just good amou AAAAAAAAAOMDAAAA 2294 1928 +Just good amou AAAAAAAABABCAAAA 2308 1807 +Just good amou AAAAAAAABAGAAAAA 2358 2308 +Just good amou AAAAAAAABCCCAAAA 1694 1863 +Just good amou AAAAAAAABECDAAAA 2361 1636 +Just good amou AAAAAAAABEMCAAAA 2644 2018 +Just good amou AAAAAAAABHACAAAA 1471 1982 +Just good amou AAAAAAAABHPCAAAA 1729 1481 +Just good amou AAAAAAAABKEBAAAA 2284 1940 +Just good amou AAAAAAAABLBDAAAA 1910 2399 +Just good amou AAAAAAAABMCBAAAA 2460 2407 +Just good amou AAAAAAAABMOBAAAA 2405 1669 +Just good amou AAAAAAAABNEBAAAA 2733 2113 +Just good amou AAAAAAAABOKDAAAA 3609 2690 +Just good amou AAAAAAAABONAAAAA 2096 2325 +Just good amou AAAAAAAABPPAAAAA 1788 1540 +Just good amou AAAAAAAACACEAAAA 1763 1240 +Just good amou AAAAAAAACAHCAAAA 1877 2693 +Just good amou AAAAAAAACALAAAAA 3249 2868 +Just good amou AAAAAAAACALBAAAA 1663 1528 +Just good amou AAAAAAAACBDBAAAA 2334 2813 +Just good amou AAAAAAAACBHBAAAA 1791 2015 +Just good amou AAAAAAAACCABAAAA 2715 2959 +Just good amou AAAAAAAACCJBAAAA 2461 1696 +Just good amou AAAAAAAACCPDAAAA 3129 2313 +Just good amou AAAAAAAACDGCAAAA 2114 1790 +Just good amou AAAAAAAACDIBAAAA 2874 3081 +Just good amou AAAAAAAACEHDAAAA 1616 1765 +Just good amou AAAAAAAACEICAAAA 2037 2461 +Just good amou AAAAAAAACFKDAAAA 2373 1698 +Just good amou AAAAAAAACGJDAAAA 2578 1814 +Just good amou AAAAAAAACGMAAAAA 2285 1553 +Just good amou AAAAAAAACHCDAAAA 2620 2504 +Just good amou AAAAAAAACIICAAAA 2800 2293 +Just good amou AAAAAAAACJBEAAAA 2718 2070 +Just good amou AAAAAAAACJGDAAAA 2153 1778 +Just good amou AAAAAAAACJNCAAAA 1482 1582 +Just good amou AAAAAAAACKBAAAAA 3122 2281 +Just good amou AAAAAAAACKCAAAAA 1664 1982 +Just good amou AAAAAAAACKHBAAAA 2222 1546 +Just good amou AAAAAAAACKHDAAAA 2312 1798 +Just good amou AAAAAAAACKKBAAAA 2463 1829 +Just good amou AAAAAAAACLDAAAAA 2523 2348 +Just good amou AAAAAAAACOLBAAAA 1630 2360 +Just good amou AAAAAAAACPFDAAAA 1863 1405 +Just good amou AAAAAAAACPKBAAAA 2088 2537 +Just good amou AAAAAAAACPLDAAAA 1982 1599 +Just good amou AAAAAAAADBECAAAA 3507 2356 +Just good amou AAAAAAAADBGAAAAA 1819 1772 +Just good amou AAAAAAAADCEEAAAA 2655 1843 +Just good amou AAAAAAAADDEAAAAA 2210 2733 +Just good amou AAAAAAAADDOCAAAA 2198 2067 +Just good amou AAAAAAAADEHCAAAA 3190 2928 +Just good amou AAAAAAAADFLDAAAA 2603 1991 +Just good amou AAAAAAAADHDAAAAA 1649 1777 +Just good amou AAAAAAAADIIAAAAA 1914 1818 +Just good amou AAAAAAAADIOAAAAA 2058 2133 +Just good amou AAAAAAAADJPBAAAA 2840 1910 +Just good amou AAAAAAAADLBEAAAA 2293 1578 +Just good amou AAAAAAAADLHBAAAA 1817 1316 +Just good amou AAAAAAAADMCCAAAA 1944 2028 +Just good amou AAAAAAAADPDBAAAA 1993 1430 +Just good amou AAAAAAAAECEDAAAA 1968 2076 +Just good amou AAAAAAAAEEOBAAAA 1992 1737 +Just good amou AAAAAAAAEEODAAAA 2938 2820 +Just good amou AAAAAAAAEFACAAAA 2213 2877 +Just good amou AAAAAAAAEGCCAAAA 2262 3212 +Just good amou AAAAAAAAEGCEAAAA 3052 2175 +Just good amou AAAAAAAAEIEDAAAA 1786 2175 +Just good amou AAAAAAAAEIHDAAAA 1938 1944 +Just good amou AAAAAAAAEINDAAAA 2402 2113 +Just good amou AAAAAAAAEKADAAAA 1327 1683 +Just good amou AAAAAAAAELKAAAAA 1817 2630 +Just good amou AAAAAAAAEMHCAAAA 2260 2878 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q22.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q22.sql.out new file mode 100644 index 0000000000000..e8a553dd5094e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q22.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +esepriableanti NULL NULL NULL 429.7808764940239 +esepriableanti importoamalg #x NULL NULL 429.7808764940239 +esepriableanti importoamalg #x fragrances NULL 429.7808764940239 +esepriableanti importoamalg #x fragrances Women 429.7808764940239 +n stbarn stbarought NULL NULL NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x NULL NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x accessories NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x accessories Men 430.0122448979592 +antiationeing NULL NULL NULL 437.03614457831327 +antiationeing amalgexporti #x NULL NULL 437.03614457831327 +antiationeing amalgexporti #x newborn NULL 437.03614457831327 +antiationeing amalgexporti #x newborn Children 437.03614457831327 +n stpriantin st NULL NULL NULL 438.77868852459017 +n stpriantin st exportiexporti #x NULL NULL 438.77868852459017 +n stpriantin st exportiexporti #x toddlers NULL 438.77868852459017 +n stpriantin st exportiexporti #x toddlers Children 438.77868852459017 +eingprically NULL NULL NULL 439.97975708502025 +eingprically amalgbrand #x NULL NULL 439.97975708502025 +eingprically amalgbrand #x semi-precious NULL 439.97975708502025 +eingprically amalgbrand #x semi-precious Jewelry 439.97975708502025 +prieingable NULL NULL NULL 440.096 +prieingable exportiunivamalg #x NULL NULL 440.096 +prieingable exportiunivamalg #x self-help NULL 440.096 +prieingable exportiunivamalg #x self-help Books 440.096 +oughteingn stationought NULL NULL NULL 440.1497975708502 +oughteingn stationought amalgscholar #x NULL NULL 440.1497975708502 +oughteingn stationought amalgscholar #x rock NULL 440.1497975708502 +oughteingn stationought amalgscholar #x rock Music 440.1497975708502 +eingationbaroughtought NULL NULL NULL 440.9721115537849 +eingationbaroughtought maxicorp #x NULL NULL 440.9721115537849 +eingationbaroughtought maxicorp #x womens watch NULL 440.9721115537849 +eingationbaroughtought maxicorp #x womens watch Jewelry 440.9721115537849 +priantibarpri NULL NULL NULL 443.45849802371544 +priantibarpri exportiimporto #x NULL NULL 443.45849802371544 +priantibarpri exportiimporto #x pants NULL 443.45849802371544 +priantibarpri exportiimporto #x pants Men 443.45849802371544 +prioughtantiation NULL NULL NULL 443.8825910931174 +prioughtantiation corpmaxi #x NULL NULL 443.8825910931174 +prioughtantiation corpmaxi #x parenting NULL 443.8825910931174 +prioughtantiation corpmaxi #x parenting Books 443.8825910931174 +eseprieingoughtought NULL NULL NULL 445.2016129032258 +eseprieingoughtought importonameless #x NULL NULL 445.2016129032258 +eseprieingoughtought importonameless #x baseball NULL 445.2016129032258 +eseprieingoughtought importonameless #x baseball Sports 445.2016129032258 +eingpriationanti NULL NULL NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x NULL NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x fiction NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x fiction Books 445.4920634920635 +antin stablecallyought NULL NULL NULL 445.54918032786884 +antin stablecallyought importoedu pack #x NULL NULL 445.54918032786884 +antin stablecallyought importoedu pack #x mens NULL 445.54918032786884 +antin stablecallyought importoedu pack #x mens Shoes 445.54918032786884 +callycallyn steing NULL NULL NULL 445.9012345679012 +callycallyn steing corpunivamalg #x NULL NULL 445.9012345679012 +callycallyn steing corpunivamalg #x mystery NULL 445.9012345679012 +callycallyn steing corpunivamalg #x mystery Books 445.9012345679012 +oughtpribarought NULL NULL NULL 446.125 +oughtpribarought exportinameless #x NULL NULL 446.125 +oughtpribarought exportinameless #x wallpaper NULL 446.125 +oughtpribarought exportinameless #x wallpaper Home 446.125 +oughtantioughtbarought NULL NULL NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x NULL NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x entertainments NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x entertainments Books 446.1847389558233 +callyoughtn stcallyought NULL NULL NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x NULL NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x pop NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x pop Music 446.43650793650795 +ationeingationableought NULL NULL NULL 446.48192771084337 +ationeingationableought namelessnameless #x NULL NULL 446.48192771084337 +ationeingationableought namelessnameless #x outdoor NULL 446.48192771084337 +ationeingationableought namelessnameless #x outdoor Sports 446.48192771084337 +priantiableese NULL NULL NULL 446.85483870967744 +priantiableese exportiedu pack #x NULL NULL 446.85483870967744 +priantiableese exportiedu pack #x kids NULL 446.85483870967744 +priantiableese exportiedu pack #x kids Shoes 446.85483870967744 +prieseeseableought NULL NULL NULL 446.9186991869919 +prieseeseableought amalgscholar #x NULL NULL 446.9186991869919 +prieseeseableought amalgscholar #x rock NULL 446.9186991869919 +prieseeseableought amalgscholar #x rock Music 446.9186991869919 +ationableoughtcallyought NULL NULL NULL 447.165991902834 +ationableoughtcallyought exportischolar #x NULL NULL 447.165991902834 +ationableoughtcallyought exportischolar #x pop NULL 447.165991902834 +ationableoughtcallyought exportischolar #x pop Music 447.165991902834 +pripricallyese NULL NULL NULL 447.2550607287449 +pripricallyese edu packimporto #x NULL NULL 447.2550607287449 +pripricallyese edu packimporto #x sports-apparel NULL 447.2550607287449 +pripricallyese edu packimporto #x sports-apparel Men 447.2550607287449 +eingableationn st NULL NULL NULL 447.3541666666667 +eingableationn st namelessmaxi #x NULL NULL 447.3541666666667 +eingableationn st namelessmaxi #x romance NULL 447.3541666666667 +eingableationn st namelessmaxi #x romance Books 447.3541666666667 +n stantin stoughtought NULL NULL NULL 448.2396694214876 +n stantin stoughtought importoscholar #x NULL NULL 448.2396694214876 +n stantin stoughtought importoscholar #x country NULL 448.2396694214876 +n stantin stoughtought importoscholar #x country Music 448.2396694214876 +n steingbaranti NULL NULL NULL 448.702479338843 +n steingbaranti amalgamalg #x NULL NULL 448.702479338843 +n steingbaranti amalgamalg #x dresses NULL 448.702479338843 +n steingbaranti amalgamalg #x dresses Women 448.702479338843 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q23a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q23a.sql.out new file mode 100644 index 0000000000000..abc7f25b4469f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q23a.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q23b.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q23b.sql.out new file mode 100644 index 0000000000000..04ba480fbec45 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q23b.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q24a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q24a.sql.out new file mode 100644 index 0000000000000..e50b231441a5d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q24a.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q24b.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q24b.sql.out new file mode 100644 index 0000000000000..e50b231441a5d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q24b.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q25.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q25.sql.out new file mode 100644 index 0000000000000..24b42a129ba4d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q25.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q26.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q26.sql.out new file mode 100644 index 0000000000000..c6760949c1f40 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q26.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAABDAAAA 84.0 18.790000 0.000000 17.280000 +AAAAAAAAAABEAAAA 76.0 145.970000 4732.320000 115.310000 +AAAAAAAAAACAAAAA 67.0 55.950000 1532.610000 48.670000 +AAAAAAAAAADBAAAA 34.333333333333336 70.963333 0.000000 20.753333 +AAAAAAAAAAEDAAAA 45.0 93.180000 0.000000 16.770000 +AAAAAAAAAAEEAAAA 46.666666666666664 73.350000 10.526667 15.433333 +AAAAAAAAAAFAAAAA 31.0 42.450000 0.000000 22.920000 +AAAAAAAAAAHAAAAA 42.0 9.700000 0.000000 0.480000 +AAAAAAAAAAHDAAAA 44.5 89.835000 0.000000 75.130000 +AAAAAAAAAAICAAAA 2.0 117.590000 68.420000 35.270000 +AAAAAAAAAAIDAAAA 11.0 6.310000 13.950000 1.510000 +AAAAAAAAAAKAAAAA 87.0 142.525000 0.000000 32.775000 +AAAAAAAAAALCAAAA 65.5 114.655000 0.000000 12.395000 +AAAAAAAAAAOAAAAA 91.0 42.190000 0.000000 30.790000 +AAAAAAAAAAPBAAAA 56.0 37.590000 0.000000 22.335000 +AAAAAAAAAAPCAAAA 35.0 49.890000 0.000000 30.430000 +AAAAAAAAABAAAAAA 87.0 77.640000 0.000000 29.500000 +AAAAAAAAABAEAAAA 79.0 34.720000 0.000000 30.900000 +AAAAAAAAABBAAAAA 16.0 53.500000 0.000000 26.515000 +AAAAAAAAABBDAAAA 79.0 91.570000 0.000000 0.910000 +AAAAAAAAABCEAAAA 100.0 106.260000 0.000000 26.560000 +AAAAAAAAABDAAAAA 58.0 3.020000 0.000000 2.080000 +AAAAAAAAABDBAAAA 21.0 72.850000 0.000000 57.550000 +AAAAAAAAABEAAAAA 71.0 70.970000 0.000000 48.250000 +AAAAAAAAABFCAAAA 33.0 284.590000 0.000000 36.990000 +AAAAAAAAABGAAAAA 54.5 77.980000 0.000000 51.590000 +AAAAAAAAABGBAAAA 84.0 18.370000 300.835000 14.035000 +AAAAAAAAABJDAAAA 42.0 41.960000 0.000000 8.810000 +AAAAAAAAABKCAAAA 27.0 62.695000 0.000000 24.770000 +AAAAAAAAABLBAAAA 62.0 112.810000 0.000000 54.795000 +AAAAAAAAABLCAAAA 42.0 206.490000 0.000000 61.940000 +AAAAAAAAABMAAAAA 45.5 158.505000 1467.465000 50.865000 +AAAAAAAAABMBAAAA 54.0 38.760000 0.000000 20.930000 +AAAAAAAAABNDAAAA 2.0 227.200000 0.000000 34.080000 +AAAAAAAAACBCAAAA 82.0 200.340000 0.000000 154.260000 +AAAAAAAAACCDAAAA 8.0 96.950000 0.000000 89.190000 +AAAAAAAAACDDAAAA 99.0 215.550000 5622.900000 66.820000 +AAAAAAAAACECAAAA 35.0 117.300000 0.000000 32.840000 +AAAAAAAAACEEAAAA 67.0 221.660000 2747.330000 164.020000 +AAAAAAAAACFAAAAA 81.0 16.420000 148.390000 2.290000 +AAAAAAAAACFBAAAA 23.0 148.690000 0.000000 96.640000 +AAAAAAAAACGAAAAA 71.5 182.975000 0.000000 89.260000 +AAAAAAAAACGDAAAA 38.0 111.060000 0.000000 43.310000 +AAAAAAAAACHBAAAA 62.0 199.820000 0.000000 67.930000 +AAAAAAAAACIDAAAA 73.0 184.700000 1925.030000 38.780000 +AAAAAAAAACJDAAAA 12.0 58.740000 319.710000 42.290000 +AAAAAAAAACLBAAAA 57.5 78.230000 391.295000 11.920000 +AAAAAAAAACLDAAAA 6.0 127.370000 0.000000 118.450000 +AAAAAAAAACMAAAAA 84.0 118.000000 0.000000 20.060000 +AAAAAAAAACNCAAAA 8.0 25.910000 99.840000 18.910000 +AAAAAAAAACODAAAA 49.0 106.120000 0.000000 6.360000 +AAAAAAAAADBBAAAA 77.0 166.050000 3605.600000 99.630000 +AAAAAAAAADCDAAAA 3.0 191.710000 0.000000 70.930000 +AAAAAAAAADDBAAAA 46.0 68.830000 0.000000 28.900000 +AAAAAAAAADEAAAAA 78.0 20.720000 0.000000 5.800000 +AAAAAAAAADEBAAAA 57.0 54.900000 0.000000 8.230000 +AAAAAAAAADFAAAAA 70.0 75.673333 777.880000 21.056667 +AAAAAAAAADFCAAAA 44.0 98.340000 182.190000 82.775000 +AAAAAAAAADFDAAAA 5.0 40.030000 0.000000 28.020000 +AAAAAAAAADGCAAAA 22.0 42.730000 0.000000 8.540000 +AAAAAAAAADKAAAAA 35.0 62.020000 0.000000 36.590000 +AAAAAAAAADMBAAAA 10.0 46.770000 0.000000 37.410000 +AAAAAAAAADNAAAAA 14.0 258.660000 0.000000 178.470000 +AAAAAAAAADNBAAAA 53.0 94.195000 0.000000 27.755000 +AAAAAAAAADNDAAAA 9.0 150.480000 0.000000 75.240000 +AAAAAAAAADOAAAAA 76.5 187.970000 310.730000 36.320000 +AAAAAAAAAEBDAAAA 7.0 68.300000 0.000000 15.700000 +AAAAAAAAAECEAAAA 81.0 241.650000 0.000000 9.660000 +AAAAAAAAAEDEAAAA 18.0 184.510000 979.740000 108.860000 +AAAAAAAAAEEDAAAA 81.0 72.050000 0.000000 56.910000 +AAAAAAAAAEGAAAAA 44.0 192.830000 0.000000 30.850000 +AAAAAAAAAEGBAAAA 39.0 6.230000 91.590000 3.050000 +AAAAAAAAAEGDAAAA 52.0 74.130000 0.000000 18.530000 +AAAAAAAAAEHAAAAA 68.0 72.870000 602.000000 59.020000 +AAAAAAAAAEHCAAAA 50.0 52.560000 0.000000 29.430000 +AAAAAAAAAEJBAAAA 66.0 66.110000 0.000000 5.940000 +AAAAAAAAAEKAAAAA 17.0 186.350000 339.590000 124.850000 +AAAAAAAAAEKCAAAA 93.0 57.210000 549.870000 6.290000 +AAAAAAAAAEKDAAAA 55.0 143.730000 0.000000 10.060000 +AAAAAAAAAELBAAAA 12.0 137.550000 0.000000 126.540000 +AAAAAAAAAEMBAAAA 100.0 52.750000 1092.980000 14.770000 +AAAAAAAAAENAAAAA 81.0 43.060000 2510.870000 32.290000 +AAAAAAAAAEPBAAAA 13.5 80.590000 0.000000 16.975000 +AAAAAAAAAFBEAAAA 93.0 115.340000 0.000000 42.670000 +AAAAAAAAAFCDAAAA 47.0 170.770000 0.000000 163.930000 +AAAAAAAAAFCEAAAA 73.0 91.970000 0.000000 59.780000 +AAAAAAAAAFDAAAAA 100.0 229.510000 2616.300000 68.850000 +AAAAAAAAAFDCAAAA 82.0 93.600000 0.000000 24.330000 +AAAAAAAAAFEEAAAA 61.0 245.950000 0.000000 199.210000 +AAAAAAAAAFFAAAAA 59.0 54.550000 555.140000 40.910000 +AAAAAAAAAFFBAAAA 65.0 142.570000 0.000000 69.850000 +AAAAAAAAAFFEAAAA 39.0 237.790000 681.580000 116.510000 +AAAAAAAAAFGCAAAA 45.0 205.590000 0.000000 47.280000 +AAAAAAAAAFHBAAAA 48.5 68.835000 0.000000 23.875000 +AAAAAAAAAFIAAAAA 72.0 84.430000 0.000000 0.000000 +AAAAAAAAAFIDAAAA 40.5 45.650000 1212.090000 17.425000 +AAAAAAAAAFJCAAAA 13.0 133.270000 0.000000 6.660000 +AAAAAAAAAFKCAAAA 5.0 178.640000 0.000000 105.390000 +AAAAAAAAAFNBAAAA 16.0 32.220000 0.000000 31.570000 +AAAAAAAAAFODAAAA 59.0 88.455000 138.365000 14.180000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q27.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q27.sql.out new file mode 100644 index 0000000000000..4e402d0379f2e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q27.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 1 50.20319480167863 76.358588 197.423228 38.217862 +AAAAAAAAAAABAAAA NULL 1 45.0 20.970000 0.000000 10.900000 +AAAAAAAAAAABAAAA TN 0 45.0 20.970000 0.000000 10.900000 +AAAAAAAAAAACAAAA NULL 1 4.0 60.970000 0.000000 35.970000 +AAAAAAAAAAACAAAA TN 0 4.0 60.970000 0.000000 35.970000 +AAAAAAAAAACDAAAA NULL 1 86.0 56.830000 0.000000 38.070000 +AAAAAAAAAACDAAAA TN 0 86.0 56.830000 0.000000 38.070000 +AAAAAAAAAADBAAAA NULL 1 61.0 40.060000 0.000000 7.210000 +AAAAAAAAAADBAAAA TN 0 61.0 40.060000 0.000000 7.210000 +AAAAAAAAAADCAAAA NULL 1 37.0 96.405000 0.000000 82.010000 +AAAAAAAAAADCAAAA TN 0 37.0 96.405000 0.000000 82.010000 +AAAAAAAAAADEAAAA NULL 1 23.0 96.010000 0.000000 57.600000 +AAAAAAAAAADEAAAA TN 0 23.0 96.010000 0.000000 57.600000 +AAAAAAAAAAEDAAAA NULL 1 64.0 10.940000 0.000000 10.390000 +AAAAAAAAAAEDAAAA TN 0 64.0 10.940000 0.000000 10.390000 +AAAAAAAAAAEEAAAA NULL 1 65.0 97.825000 0.000000 24.320000 +AAAAAAAAAAEEAAAA TN 0 65.0 97.825000 0.000000 24.320000 +AAAAAAAAAAFAAAAA NULL 1 71.0 88.080000 0.000000 10.560000 +AAAAAAAAAAFAAAAA TN 0 71.0 88.080000 0.000000 10.560000 +AAAAAAAAAAFCAAAA NULL 1 21.0 72.140000 0.000000 54.725000 +AAAAAAAAAAFCAAAA TN 0 21.0 72.140000 0.000000 54.725000 +AAAAAAAAAAGBAAAA NULL 1 23.0 97.980000 0.000000 45.070000 +AAAAAAAAAAGBAAAA TN 0 23.0 97.980000 0.000000 45.070000 +AAAAAAAAAAGCAAAA NULL 1 30.0 62.280000 0.000000 9.340000 +AAAAAAAAAAGCAAAA TN 0 30.0 62.280000 0.000000 9.340000 +AAAAAAAAAAHAAAAA NULL 1 30.0 91.910000 0.000000 52.380000 +AAAAAAAAAAHAAAAA TN 0 30.0 91.910000 0.000000 52.380000 +AAAAAAAAAAHBAAAA NULL 1 76.0 30.060000 345.610000 26.750000 +AAAAAAAAAAHBAAAA TN 0 76.0 30.060000 345.610000 26.750000 +AAAAAAAAAAHDAAAA NULL 1 60.0 49.390000 0.000000 19.155000 +AAAAAAAAAAHDAAAA TN 0 60.0 49.390000 0.000000 19.155000 +AAAAAAAAAAIAAAAA NULL 1 65.0 115.230000 0.000000 79.555000 +AAAAAAAAAAIAAAAA TN 0 65.0 115.230000 0.000000 79.555000 +AAAAAAAAAAICAAAA NULL 1 NULL NULL 262.480000 NULL +AAAAAAAAAAICAAAA TN 0 NULL NULL 262.480000 NULL +AAAAAAAAAAJCAAAA NULL 1 7.0 111.745000 0.000000 82.515000 +AAAAAAAAAAJCAAAA TN 0 7.0 111.745000 0.000000 82.515000 +AAAAAAAAAAKAAAAA NULL 1 31.5 51.350000 0.000000 36.555000 +AAAAAAAAAAKAAAAA TN 0 31.5 51.350000 0.000000 36.555000 +AAAAAAAAAAKBAAAA NULL 1 3.0 146.600000 0.000000 105.550000 +AAAAAAAAAAKBAAAA TN 0 3.0 146.600000 0.000000 105.550000 +AAAAAAAAAAKDAAAA NULL 1 69.0 34.660000 0.000000 11.090000 +AAAAAAAAAAKDAAAA TN 0 69.0 34.660000 0.000000 11.090000 +AAAAAAAAAALAAAAA NULL 1 97.0 14.270000 0.000000 12.410000 +AAAAAAAAAALAAAAA TN 0 97.0 14.270000 0.000000 12.410000 +AAAAAAAAAAMBAAAA NULL 1 68.5 70.250000 0.000000 34.085000 +AAAAAAAAAAMBAAAA TN 0 68.5 70.250000 0.000000 34.085000 +AAAAAAAAAAMCAAAA NULL 1 51.5 73.135000 0.000000 25.570000 +AAAAAAAAAAMCAAAA TN 0 51.5 73.135000 0.000000 25.570000 +AAAAAAAAAANAAAAA NULL 1 50.5 29.315000 9.580000 15.805000 +AAAAAAAAAANAAAAA TN 0 50.5 29.315000 9.580000 15.805000 +AAAAAAAAAAOCAAAA NULL 1 1.0 74.630000 0.000000 19.400000 +AAAAAAAAAAOCAAAA TN 0 1.0 74.630000 0.000000 19.400000 +AAAAAAAAAAODAAAA NULL 1 66.33333333333333 52.823333 1793.560000 28.406667 +AAAAAAAAAAODAAAA TN 0 66.33333333333333 52.823333 1793.560000 28.406667 +AAAAAAAAAAPBAAAA NULL 1 17.0 167.070000 0.000000 40.090000 +AAAAAAAAAAPBAAAA TN 0 17.0 167.070000 0.000000 40.090000 +AAAAAAAAABAAAAAA NULL 1 79.0 48.110000 0.000000 14.430000 +AAAAAAAAABAAAAAA TN 0 79.0 48.110000 0.000000 14.430000 +AAAAAAAAABAEAAAA NULL 1 16.5 26.370000 0.000000 9.325000 +AAAAAAAAABAEAAAA TN 0 16.5 26.370000 0.000000 9.325000 +AAAAAAAAABCBAAAA NULL 1 32.0 98.600000 0.000000 46.340000 +AAAAAAAAABCBAAAA TN 0 32.0 98.600000 0.000000 46.340000 +AAAAAAAAABFCAAAA NULL 1 24.0 101.420000 0.000000 3.040000 +AAAAAAAAABFCAAAA TN 0 24.0 101.420000 0.000000 3.040000 +AAAAAAAAABFEAAAA NULL 1 34.5 42.490000 131.055000 15.505000 +AAAAAAAAABFEAAAA TN 0 34.5 42.490000 131.055000 15.505000 +AAAAAAAAABGAAAAA NULL 1 57.666666666666664 94.343333 569.626667 56.296667 +AAAAAAAAABGAAAAA TN 0 57.666666666666664 94.343333 569.626667 56.296667 +AAAAAAAAABHCAAAA NULL 1 58.0 38.370000 0.000000 16.880000 +AAAAAAAAABHCAAAA TN 0 58.0 38.370000 0.000000 16.880000 +AAAAAAAAABHDAAAA NULL 1 43.0 10.290000 0.000000 3.800000 +AAAAAAAAABHDAAAA TN 0 43.0 10.290000 0.000000 3.800000 +AAAAAAAAABIBAAAA NULL 1 60.25 74.462500 0.000000 58.247500 +AAAAAAAAABIBAAAA TN 0 60.25 74.462500 0.000000 58.247500 +AAAAAAAAABJDAAAA NULL 1 64.5 28.405000 0.000000 22.080000 +AAAAAAAAABJDAAAA TN 0 64.5 28.405000 0.000000 22.080000 +AAAAAAAAABKCAAAA NULL 1 69.0 44.880000 518.585000 18.855000 +AAAAAAAAABKCAAAA TN 0 69.0 44.880000 518.585000 18.855000 +AAAAAAAAABLBAAAA NULL 1 18.0 43.495000 0.000000 24.660000 +AAAAAAAAABLBAAAA TN 0 18.0 43.495000 0.000000 24.660000 +AAAAAAAAABLCAAAA NULL 1 6.0 102.740000 0.000000 90.410000 +AAAAAAAAABLCAAAA TN 0 6.0 102.740000 0.000000 90.410000 +AAAAAAAAABMDAAAA NULL 1 50.5 32.745000 0.000000 12.005000 +AAAAAAAAABMDAAAA TN 0 50.5 32.745000 0.000000 12.005000 +AAAAAAAAABNAAAAA NULL 1 85.0 31.880000 0.000000 5.100000 +AAAAAAAAABNAAAAA TN 0 85.0 31.880000 0.000000 5.100000 +AAAAAAAAABOCAAAA NULL 1 60.0 113.590000 51.520000 1.130000 +AAAAAAAAABOCAAAA TN 0 60.0 113.590000 51.520000 1.130000 +AAAAAAAAABPAAAAA NULL 1 55.0 89.150000 3442.050000 76.440000 +AAAAAAAAABPAAAAA TN 0 55.0 89.150000 3442.050000 76.440000 +AAAAAAAAABPBAAAA NULL 1 80.0 16.010000 0.000000 9.440000 +AAAAAAAAABPBAAAA TN 0 80.0 16.010000 0.000000 9.440000 +AAAAAAAAABPDAAAA NULL 1 73.0 112.940000 2248.960000 99.380000 +AAAAAAAAABPDAAAA TN 0 73.0 112.940000 2248.960000 99.380000 +AAAAAAAAACAAAAAA NULL 1 61.0 101.820000 0.000000 90.610000 +AAAAAAAAACAAAAAA TN 0 61.0 101.820000 0.000000 90.610000 +AAAAAAAAACACAAAA NULL 1 86.0 101.500000 0.000000 57.850000 +AAAAAAAAACACAAAA TN 0 86.0 101.500000 0.000000 57.850000 +AAAAAAAAACADAAAA NULL 1 65.0 97.210000 0.000000 83.595000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q28.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q28.sql.out new file mode 100644 index 0000000000000..5d3f3fa3c8602 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q28.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +78.045281 36383 9236 69.528580 35193 6542 133.847037 28274 9714 81.911887 31756 7687 61.160300 36338 8603 39.282627 29915 5210 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q29.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q29.sql.out new file mode 100644 index 0000000000000..467b5c3ba3eaf --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q29.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAADIFDAAAA Now christian papers believe very major, new branches. Annual wars include harshly so-called sites. AAAAAAAAHAAAAAAA ation 11 10 13 +AAAAAAAANNBEAAAA Old forces shall not think more than foreign earnings. Controls could carry almos AAAAAAAACAAAAAAA able 56 25 10 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q3.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q3.sql.out new file mode 100644 index 0000000000000..65e15769d5b0f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q3.sql.out @@ -0,0 +1,88 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1998 1004001 edu packamalg #x 65716.37 +1998 10001008 amalgunivamalg #x 34140.78 +1998 8012001 importomaxi #x 32669.34 +1998 2004001 edu packimporto #x 25130.97 +1998 10003014 exportiunivamalg #x 23720.25 +1998 5004001 edu packscholar #x 23713.55 +1998 9002008 importomaxi #x 22002.12 +1998 3003001 exportiexporti #x 21596.96 +1998 8014004 edu packmaxi #x 20442.12 +1998 9009005 maximaxi #x 19866.63 +1998 3002001 importoexporti #x 17347.94 +1998 3001001 amalgexporti #x 16882.10 +1998 2003001 exportiimporto #x 13095.00 +1998 8010001 univmaxi #x 12408.22 +1998 8007001 brandnameless #x 12021.05 +1998 6005005 scholarcorp #x 10366.33 +1998 8013008 exportimaxi #x 10008.30 +1998 10008008 namelessunivamalg #x 7909.24 +1998 1003002 exportiamalg #x 5046.65 +1999 1004001 edu packamalg #x 72111.20 +1999 8012001 importomaxi #x 45932.95 +1999 9009005 maximaxi #x 32382.90 +1999 1003002 exportiamalg #x 28586.32 +1999 8007001 brandnameless #x 27105.57 +1999 9002008 importomaxi #x 26746.44 +1999 5004001 edu packscholar #x 25906.67 +1999 8010001 univmaxi #x 23297.75 +1999 8013008 exportimaxi #x 20896.22 +1999 2004001 edu packimporto #x 18025.93 +1999 10003014 exportiunivamalg #x 16211.31 +1999 8014004 edu packmaxi #x 15207.49 +1999 3001001 amalgexporti #x 13536.73 +1999 10001008 amalgunivamalg #x 12980.62 +1999 3003001 exportiexporti #x 12753.61 +1999 10008008 namelessunivamalg #x 12446.57 +1999 2003001 exportiimporto #x 11284.94 +1999 3002001 importoexporti #x 9820.44 +1999 6005005 scholarcorp #x 5487.63 +2000 3002001 importoexporti #x 49567.04 +2000 1004001 edu packamalg #x 35173.68 +2000 6006004 corpcorp #x 31206.02 +2000 1003002 exportiamalg #x 30500.97 +2000 6005005 scholarcorp #x 30270.09 +2000 9001012 amalgmaxi #x 29107.66 +2000 8012001 importomaxi #x 26610.58 +2000 5004001 edu packscholar #x 26200.25 +2000 10003014 exportiunivamalg #x 23018.57 +2000 3001001 amalgexporti #x 20499.18 +2000 2004001 edu packimporto #x 19982.58 +2000 10015010 importoedu pack #x 18697.08 +2000 8007001 brandnameless #x 17315.93 +2000 8014004 edu packmaxi #x 16235.88 +2000 3003001 exportiexporti #x 14272.86 +2000 8013008 exportimaxi #x 10570.32 +2000 9002008 importomaxi #x 10262.91 +2001 1003002 exportiamalg #x 36559.22 +2001 9002008 importomaxi #x 25301.41 +2001 8013008 exportimaxi #x 21069.43 +2001 6005003 importoamalg #x 20705.88 +2001 9001012 amalgmaxi #x 18795.96 +2001 8012001 exportiedu pack #x 18477.27 +2001 6006004 corpcorp #x 18283.57 +2001 9003010 exportimaxi #x 17793.12 +2001 3002001 scholarmaxi #x 17206.42 +2001 10003014 exportiunivamalg #x 15781.84 +2001 10001003 edu packamalg #x 15683.42 +2001 8014004 edu packmaxi #x 13108.20 +2001 10015010 importoedu pack #x 11683.34 +2001 2002002 importoimporto #x 8532.67 +2002 1003002 exportiamalg #x 53079.32 +2002 9002008 importomaxi #x 39552.73 +2002 3002001 scholarmaxi #x 38802.50 +2002 9003010 exportimaxi #x 36257.87 +2002 5002001 importoscholar #x 36116.22 +2002 10015010 amalgunivamalg #x 28522.27 +2002 8012001 exportiedu pack #x 27777.47 +2002 8013008 exportimaxi #x 23919.27 +2002 2002002 importoimporto #x 21849.78 +2002 8014004 edu packmaxi #x 19276.30 +2002 10001003 edu packamalg #x 13837.38 +2002 10003014 exportiunivamalg #x 13106.68 +2002 9015001 scholarunivamalg #x 11700.36 +2002 6005003 importoamalg #x 6367.68 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q30.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q30.sql.out new file mode 100644 index 0000000000000..2a298679ad558 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q30.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAABEBBAAA Sir Carlton Miles N 23 9 1957 MALI NULL Carlton.Miles@4DbIUoznbr.org 2452588 12916.86 +AAAAAAAAABKGBAAA Miss Sharee Stevens Y 15 8 1992 SLOVAKIA NULL Sharee.Stevens@Z.edu 2452492 2480.80 +AAAAAAAAAEIPAAAA Mr. Chris Ryan Y 23 4 1931 TURKMENISTAN NULL Chris.Ryan@QlH9G0fkAR5.org 2452443 3469.95 +AAAAAAAAAHNKAAAA Sir Scott Garcia N 19 10 1952 WALLIS AND FUTUNA NULL Scott.Garcia@3pd6mnJYbKxb.org 2452611 2944.24 +AAAAAAAAAJDABAAA Sir Gerald Monroe Y 19 5 1985 LIECHTENSTEIN NULL Gerald.Monroe@opYT.org 2452632 4030.41 +AAAAAAAAAJJBAAAA Sir Jose Garcia Y 11 6 1977 AMERICAN SAMOA NULL Jose.Garcia@n1VrEIOg4f.com 2452406 4131.05 +AAAAAAAAALNKAAAA Dr. Wanda Devries N 6 2 1953 YEMEN NULL Wanda.Devries@lzbOovP.com 2452411 3197.25 +AAAAAAAAANPAAAAA NULL NULL Sanders NULL 12 10 NULL NULL NULL NULL 2452646 2091.90 +AAAAAAAAAOKPAAAA Mr. Brandon Foster N 28 4 1992 FRENCH POLYNESIA NULL Brandon.Foster@vKi8eFToOZHK.org 2452527 3706.08 +AAAAAAAABBEDAAAA Mr. James Pantoja Y 5 12 1982 NEPAL NULL James.Pantoja@LTZLj3ddIvIG0.edu 2452503 5668.25 +AAAAAAAABBIHAAAA Ms. Sherry Kennedy Y 3 8 1984 MOZAMBIQUE NULL Sherry.Kennedy@9hhBNI6.edu 2452481 1561.14 +AAAAAAAABBJMAAAA NULL Paul Lindsey N NULL 10 NULL NULL NULL Paul.Lindsey@bQo8Zt9XStF.edu NULL 3139.50 +AAAAAAAABBNFBAAA Mrs. Nellie Johnson Y 14 6 1962 BAHRAIN NULL Nellie.Johnson@U.com 2452411 1609.90 +AAAAAAAABDKOAAAA Dr. Vernita Bennett Y 12 3 1955 NICARAGUA NULL Vernita.Bennett@u2QKJHt.com 2452352 11315.85 +AAAAAAAABEJLAAAA Mr. Abel Lucero N 15 9 1930 CAPE VERDE NULL Abel.Lucero@txdlc3sekSyj7mLokv.org 2452461 4524.56 +AAAAAAAABFFHBAAA Mrs. Arlene Parrish N 18 9 1957 MALAWI NULL Arlene.Parrish@m5lX3mTVhbxAEz.org 2452433 3184.72 +AAAAAAAABGCFBAAA Sir Alberto Whitney Y 20 5 1946 MALAYSIA NULL Alberto.Whitney@En6lzc8TlV7vF.org 2452496 3806.56 +AAAAAAAABHAGBAAA Mrs. Minnie Thompson N 1 8 1976 NEW ZEALAND NULL Minnie.Thompson@7u.org 2452558 1871.31 +AAAAAAAABLMCAAAA Dr. Frank Price N 13 2 1962 SYRIAN ARAB REPUBLIC NULL Frank.Price@KO.org 2452372 1779.84 +AAAAAAAABNEGAAAA Ms. Laurel Olson Y 24 6 1960 BANGLADESH NULL Laurel.Olson@92YpV3hofMbNz.edu 2452508 4912.49 +AAAAAAAABNMIAAAA Mrs. Judi Lopes Y 21 2 1943 NICARAGUA NULL Judi.Lopes@DYAyBUb1E.org 2452297 8335.86 +AAAAAAAABOGDAAAA Ms. Laura Mcfadden N 29 3 1964 GREECE NULL Laura.Mcfadden@Ks.com 2452364 3942.35 +AAAAAAAABPECAAAA Mr. William Hoffman N 21 4 1991 PERU NULL William.Hoffman@m8E1N4.com 2452321 4550.73 +AAAAAAAACGHDAAAA Mr. Ricky Marshall N 11 7 1979 NAMIBIA NULL Ricky.Marshall@tJKkX.edu 2452610 10400.46 +AAAAAAAACIKBAAAA Ms. Virginia Lawson N 26 11 1929 PHILIPPINES NULL Virginia.Lawson@bdtmjhsnqQZx.edu 2452490 2164.20 +AAAAAAAACILFBAAA NULL Frederick NULL N 23 11 1954 NULL NULL Frederick.Lambert@BNHNh23xF7qIUJ30.edu NULL 3849.92 +AAAAAAAACLNHBAAA Dr. Gary Perez N 21 4 1992 MACAO NULL Gary.Perez@5dGey4.com 2452300 9647.56 +AAAAAAAACOEAAAAA Dr. David Saunders Y 6 12 1991 HAITI NULL David.Saunders@APTXJyYNgVa.edu 2452482 8357.76 +AAAAAAAADBIHAAAA Mr. John Sturm Y 5 8 1973 GUAM NULL John.Sturm@PnUSI7.com 2452457 7161.74 +AAAAAAAADCCCBAAA Ms. Antonia Miller N 1 5 1992 PALAU NULL Antonia.Miller@CsPe3fE38F.com 2452635 2688.66 +AAAAAAAADGIGBAAA Miss Margarita Park Y 20 1 1947 LITHUANIA NULL Margarita.Park@qb0FrD2Pk.org 2452462 1557.37 +AAAAAAAADGMJAAAA Mrs. Karen Olson Y 18 11 1974 ARGENTINA NULL Karen.Olson@Qe3bBGr.com 2452596 11849.40 +AAAAAAAADGOIAAAA NULL NULL NULL NULL 2 NULL 1977 NULL NULL Kathleen.Bell@B0Ze.com 2452322 3979.64 +AAAAAAAADHFBAAAA Mr. William Lopez N 12 1 1967 MALAYSIA NULL William.Lopez@9JMTYo5O22ZFskMMybp.org 2452435 2302.41 +AAAAAAAADJJMAAAA Mr. James Cooper Y 26 4 1982 TUNISIA NULL James.Cooper@KH.com 2452365 3388.66 +AAAAAAAADJPEBAAA Ms. Lizzie Neal Y 16 2 1939 MAURITIUS NULL Lizzie.Neal@ru9qksDH5rgLsRn.edu 2452433 1915.72 +AAAAAAAADKJCAAAA Dr. Daniel Green N 29 8 1970 AMERICAN SAMOA NULL Daniel.Green@GPktIG5Y.org 2452421 5675.02 +AAAAAAAADMFKAAAA Mr. Ray Gonzales N 22 2 1930 TURKMENISTAN NULL Ray.Gonzales@kH8kp.com 2452378 2294.25 +AAAAAAAADMHDAAAA Mr. Lee Stanley Y 9 3 1959 CZECH REPUBLIC NULL Lee.Stanley@rONmrEDug8q.com 2452451 1912.92 +AAAAAAAADMONAAAA Sir Joshua West Y 15 3 1987 ALAND ISLANDS NULL Joshua.West@5q4JZHUuo9h0e1ol.org 2452545 2870.00 +AAAAAAAADOBAAAAA Dr. Paul Yeager N 17 11 1946 EGYPT NULL Paul.Yeager@PKzHg.edu 2452571 3050.46 +AAAAAAAAEADIAAAA Dr. Gregory Carey Y 6 1 1937 SUDAN NULL Gregory.Carey@495UmqFpU0My0GB8.edu 2452330 5133.18 +AAAAAAAAECBEAAAA Ms. Megan Wilson N 1 12 1935 COMOROS NULL Megan.Wilson@mi.com 2452501 1622.24 +AAAAAAAAECIEAAAA Miss Danille Sanders Y 15 3 1942 OMAN NULL Danille.Sanders@dnjPPYBZQ.com 2452566 5474.66 +AAAAAAAAEEBEBAAA Dr. Jamie Jackson Y 21 11 1933 POLAND NULL Jamie.Jackson@dLUBOuU.com 2452365 1761.68 +AAAAAAAAEEGMAAAA Dr. NULL NULL N 5 NULL NULL NULL NULL NULL 2452438 2046.20 +AAAAAAAAEEJAAAAA Dr. Rhonda Anderson N 27 12 1942 ISLE OF MAN NULL Rhonda.Anderson@P2VN7VQxGjj.com 2452394 2888.50 +AAAAAAAAEENFAAAA Dr. James Brown N 27 9 1971 NEW CALEDONIA NULL James.Brown@dzkgBG43cs.org 2452466 2474.24 +AAAAAAAAEGDDBAAA Mr. Timothy Barajas N 8 10 1976 MAURITANIA NULL Timothy.Barajas@FFhEdu3MO4n.com 2452594 2209.06 +AAAAAAAAEGEGAAAA Mrs. Arthur Kirk Y 24 8 1972 SLOVAKIA NULL Arthur.Kirk@3eFiyMXiuRCqt1ofD.com 2452286 3189.74 +AAAAAAAAEGGKAAAA Miss Johanna Moses Y 3 5 1971 R�UNION NULL Johanna.Moses@AhPc5dGr7FubqU1Lyj.com 2452443 2842.06 +AAAAAAAAEGPPAAAA Mr. Michael Pringle Y 1 10 1990 JERSEY NULL Michael.Pringle@noEgytx7nOED.edu 2452398 1870.56 +AAAAAAAAEHFCBAAA Mrs. Lillian Yazzie N 14 1 1936 GEORGIA NULL Lillian.Yazzie@Hzg3QQh.com 2452555 1758.12 +AAAAAAAAEHLDAAAA Ms. NULL Brown NULL NULL 3 NULL NULL NULL NULL NULL 4483.20 +AAAAAAAAEHLNAAAA Mr. Gregory Wester Y 9 3 1946 SPAIN NULL Gregory.Wester@ko.com 2452535 2303.28 +AAAAAAAAEKCJAAAA Mrs. Keri Lawrence Y 31 10 1955 GUERNSEY NULL Keri.Lawrence@yIBxLUPgozYINi.com 2452523 1523.98 +AAAAAAAAEMELAAAA Dr. Steven Parker N 11 9 1936 MADAGASCAR NULL Steven.Parker@E6kIVz3.org 2452372 2106.30 +AAAAAAAAEMHBAAAA Mr. Brandon Ray Y 13 8 1976 ANTARCTICA NULL Brandon.Ray@OP6YgS6SQnuykF.org 2452290 2258.53 +AAAAAAAAFCPEBAAA Dr. Kristi Brennan Y NULL 11 NULL NULL NULL NULL 2452349 6815.00 +AAAAAAAAFDBAAAAA Mrs. Virginia Sims Y 16 4 1969 BULGARIA NULL Virginia.Sims@3qndx2y.edu 2452302 6723.52 +AAAAAAAAFGICAAAA Dr. Will Isbell N 1 10 1975 ERITREA NULL Will.Isbell@100I71HVxMaaTVZ5MH2m.org 2452482 1765.89 +AAAAAAAAFKGEBAAA Dr. Dorothy Hendricks N 14 9 1947 NAMIBIA NULL Dorothy.Hendricks@7X7OXy1xAs7hVN.org 2452562 2333.76 +AAAAAAAAFKNEAAAA Ms. Ruth Catron Y 2 3 1983 SAINT LUCIA NULL Ruth.Catron@g4krLkSRUFX60t4P.edu 2452401 1873.40 +AAAAAAAAFKOHBAAA Ms. Annie Stevens N 12 2 1933 KENYA NULL Annie.Stevens@e3vFdRXUEQ.org 2452504 4229.82 +AAAAAAAAFMDEAAAA Sir Dennis Mayfield N 5 1 1933 SOMALIA NULL Dennis.Mayfield@9DPHTIUxRlxd.edu 2452535 2849.00 +AAAAAAAAFMIPAAAA Dr. Michael Moran N 1 2 1940 SAINT HELENA NULL Michael.Moran@3.edu 2452411 3869.60 +AAAAAAAAFPKCBAAA Dr. William Terry Y 10 3 1991 GUATEMALA NULL William.Terry@jKyKRKfrxH.org 2452325 3550.05 +AAAAAAAAFPPGAAAA Ms. Margie Lee Y 22 10 1933 GUINEA NULL Margie.Lee@Tg2pE7.com 2452419 3288.60 +AAAAAAAAGBOAAAAA Mr. Micheal NULL NULL NULL NULL 1991 NULL NULL Micheal.Holland@4rhkIkNJy6fSU.edu 2452530 2676.00 +AAAAAAAAGIEDAAAA Miss Adriana Maxfield Y 6 1 1948 MONGOLIA NULL Adriana.Maxfield@Na4ize7RHB.com 2452415 4872.96 +AAAAAAAAGJJEAAAA Miss NULL Barnard N NULL NULL 1987 NULL NULL NULL NULL 1628.77 +AAAAAAAAGMJPAAAA Miss Deena Ferguson Y 8 9 1986 PHILIPPINES NULL Deena.Ferguson@19lazfjoSTXBorQ.edu 2452301 5713.74 +AAAAAAAAHANIAAAA Mr. Robert Rogers Y 5 7 1990 SOUTH AFRICA NULL Robert.Rogers@m77qeFKJfIO3rugyC.org 2452456 1638.00 +AAAAAAAAHCHKAAAA Dr. Harry Bearden Y 10 2 1957 CHINA NULL Harry.Bearden@kNxzQ3SYEmp.com 2452533 6455.11 +AAAAAAAAHHAKAAAA Dr. Jeremy Richter Y 8 8 1951 SOUTH AFRICA NULL Jeremy.Richter@hCAQGRgTCrU.edu 2452496 3764.21 +AAAAAAAAHIMLAAAA Dr. Kathleen Floyd N 16 12 1983 TURKMENISTAN NULL Kathleen.Floyd@ACpkOV2nlHDL.edu 2452644 5796.90 +AAAAAAAAHLMCAAAA Miss Adriana Case N 16 3 1959 GREECE NULL Adriana.Case@aRv.com 2452408 1740.52 +AAAAAAAAHMFBBAAA Dr. Katheryn White Y 21 7 1983 CAPE VERDE NULL Katheryn.White@dqdzeup0a7TYhnOB.edu 2452417 20518.95 +AAAAAAAAHMNFAAAA Dr. Stephen Belanger Y 10 11 1951 GUADELOUPE NULL Stephen.Belanger@emUp.edu 2452340 5734.80 +AAAAAAAAHNAOAAAA Mrs. Anita Gardner Y 27 6 1979 GEORGIA NULL Anita.Gardner@g.org 2452557 2921.42 +AAAAAAAAHOIBAAAA Miss Vera Forte N 20 9 1941 ZAMBIA NULL Vera.Forte@XxMnMUGRS.edu 2452549 2533.00 +AAAAAAAAHOOGAAAA Sir Jerry Rader Y 31 1 1948 PAPUA NEW GUINEA NULL Jerry.Rader@FAhBv4pGRIGB.com 2452327 3590.16 +AAAAAAAAHPGNAAAA Dr. Renae Richardson Y 4 8 1927 SYRIAN ARAB REPUBLIC NULL Renae.Richardson@eGnpquq6hFfm20r.com 2452426 6468.92 +AAAAAAAAIAGBBAAA Ms. Alexis Evans N 3 12 1966 LITHUANIA NULL Alexis.Evans@2cqJZnt7z8.edu 2452596 2043.99 +AAAAAAAAICGBAAAA Miss Gertrude Rodriguez Y 10 4 1932 SAN MARINO NULL Gertrude.Rodriguez@lj4qAeL9afqB9jS.com 2452447 2865.39 +AAAAAAAAIDFJAAAA Dr. Dixie Brown N 1 5 1983 FRENCH POLYNESIA NULL Dixie.Brown@PrD49KMoXd8SEMuCS.com 2452595 2135.70 +AAAAAAAAIJCHBAAA Dr. Karen Cosby Y 17 9 1963 IRAQ NULL Karen.Cosby@H17xf1rPlPtkdKTr.com 2452368 3731.50 +AAAAAAAAIJDEBAAA Mrs. Elisa Baldwin Y 14 5 1983 SWEDEN NULL Elisa.Baldwin@S.edu 2452579 3168.75 +AAAAAAAAILKLAAAA Ms. Janice Cannon N 11 12 1983 ARGENTINA NULL Janice.Cannon@lmR.org 2452353 2810.73 +AAAAAAAAINLKAAAA Ms. Mandy Anderson N 30 6 1958 MOZAMBIQUE NULL Mandy.Anderson@ONxvienlYHJpCe.org 2452539 9893.12 +AAAAAAAAIOGPAAAA Ms. Kathryn Cooper N 27 1 1982 ARGENTINA NULL Kathryn.Cooper@VBO8.com 2452457 1870.56 +AAAAAAAAJDKCAAAA Dr. Curtis Mcguire Y 10 5 1974 ETHIOPIA NULL Curtis.Mcguire@Vex25USDKvv.org 2452411 4900.15 +AAAAAAAAJFEFBAAA Ms. Nancy Hampton N 24 3 1946 WESTERN SAHARA NULL Nancy.Hampton@K8CudFMgtgyGyzS.com 2452309 6277.08 +AAAAAAAAJGNBAAAA Sir Robert Johnston N 14 5 1956 FAROE ISLANDS NULL Robert.Johnston@ccQCj7j.edu 2452631 2383.74 +AAAAAAAAJHHEBAAA Miss Else Carter Y 19 8 1963 NORFOLK ISLAND NULL Else.Carter@e.com 2452580 4210.60 +AAAAAAAAJHIBAAAA Mr. Edwin Christensen N 23 6 1947 URUGUAY NULL Edwin.Christensen@3fBqkiUX.com 2452485 5984.00 +AAAAAAAAJJFBBAAA Sir Michael Toney Y 2 6 1981 COSTA RICA NULL Michael.Toney@Oe1SH.edu 2452306 3453.63 +AAAAAAAAJKDCBAAA Dr. Robert James Y 8 11 1929 LEBANON NULL Robert.James@DSVGYuMQLZNo9oga1.edu 2452306 1922.62 +AAAAAAAAJLAGBAAA Mrs. Isabel Barber N 24 4 1964 ROMANIA NULL Isabel.Barber@0RSzJgpalSYmAoYJgnL.org 2452435 2562.91 +AAAAAAAAJLICBAAA Ms. Fumiko Ebert N 5 12 1956 NAURU NULL Fumiko.Ebert@8kz.com 2452309 2408.38 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q31.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q31.sql.out new file mode 100644 index 0000000000000..7ef2fb4948563 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q31.sql.out @@ -0,0 +1,66 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Bedford city 2000 1.41895245428186244343 1.28647520475423955859 1.98397166014536480387 1.22052485500235683695 +Blaine County 2000 0.90103169326915114550 0.83066560812940517400 5.91832927336153981754 2.20534708400177561487 +Boone County 2000 1.10318140625440703802 1.01388129938545215917 2.93583104434541244580 0.81595222675278264090 +Bristol County 2000 2.03252306215265583168 0.84389707668347883239 1.88436946252627433938 0.50855932416819944462 +Buffalo County 2000 0.79851677786134640592 0.66608776395688494889 4.12963020014993273566 1.41417145222467738551 +Carlisle County 2000 1.69492598618078795489 1.25168931735787673049 1.50718182376907682203 0.88181897085363449075 +Carroll County 2000 1.35087769364534219248 1.29257051856006991407 2.20994529474585418644 1.48849785082428358420 +Cedar County 2000 1.73903789422605330870 0.46305877715234029287 8.77007422523370529780 3.61252572439703538764 +Clinton County 2000 1.74067151347304310575 1.41218893174550937759 1.44667681199587337982 1.25445563958487852628 +Colbert County 2000 1.18024333271178132777 0.83788572603247302022 0.54236640028023008306 0.42663333520116081271 +Crawford County 2000 1.31661755312891088929 0.94956543151027392277 2.05054841119621802578 1.32014748612874091613 +Cumberland County 2000 1.08524453978148728453 0.80144708440874338843 1.60419019384387788968 1.54733087343509046916 +Dillon County 2000 0.32644424794620832608 0.12281769967769250465 2.54843886762550432202 2.36794563665865531250 +Drew County 2000 1.33752592432768148542 0.42051684931974493150 4.42925583278116865936 3.59005466753091100322 +El Paso County 2000 1.33106683366793877022 0.52026046839940548659 6.85897871779069724650 3.52274370756107989319 +Escambia County 2000 1.29140013472373800933 1.22312591570259586531 4.94949857002487942979 1.90881493105514558681 +Essex County 2000 0.91140422255526176760 0.80397981840092167933 2.40581838121906819055 1.51850239999231343611 +Forsyth County 2000 0.68611318852553022140 0.65587408738915750878 3.37787340847752340465 1.32466135376522054649 +Franklin County 2000 1.23346105169430839663 1.16004116355148303419 3.11011280618208475383 1.72852883588875015022 +Fremont County 2000 2.08342397478237427534 0.99231216361555377660 4.64450632140050612837 1.66543397250235949142 +Gaines County 2000 0.18699408635518783605 0.15864149496085374898 4.38012199409197995027 3.26392730974685542866 +Gallatin County 2000 0.57267047178378396314 0.42960913482855774228 5.40426025792003986274 2.60129747557845045914 +Hale County 2000 1.13181285759861396698 0.37084472158598998345 2.76039662764995099764 2.04880905907673376536 +Howard County 2000 0.75689303418215066506 0.40416603686332285826 5.20017335230756289550 3.93113017208635232323 +Jennings County 2000 2.78984076175313855515 2.07599697431575122264 3.44972481162691656332 2.44378922187060355318 +Johnson County 2000 2.57795641894299816468 0.91616273582603658596 1.74313979276860348710 1.18351065602569093409 +Langlade County 2000 1.68676813359812642201 0.96428449439984805852 2.54522036089486624332 1.23454212891942526190 +Lee County 2000 1.30443690529253126871 0.72035286226320500421 2.23150088973480218462 1.84186089836441132231 +Limestone County 2000 3.32976919500999168869 0.75905050855357115484 1.82617549463146988019 1.71235192999491983271 +Lonoke County 2000 2.08982990182342723842 1.10340085014357551300 1.15539868818698913544 0.99289022395794532472 +Milam County 2000 2.28508195011379140646 0.46652927795640169049 0.78408081688146536737 0.47493776386092038861 +Mineral County 2000 3.34300539922925594005 2.45993211849872338822 2.20409211845792015572 1.08747970913885456325 +Mitchell County 2000 1.67453892386598503677 1.34365298077586851819 1.23346415770637208954 0.95073923197301166934 +Montgomery County 2000 1.56847414391333207258 0.83177598227578637405 2.13678203835469561935 2.10257648921068871721 +Moore County 2000 2.15694071746428807771 0.40796143296369785394 1.74966386964488096764 1.16471835770020727271 +Ottawa County 2000 1.75323872261604668329 0.83468698542567460304 1.61973935146544593444 1.42351390679742385404 +Pamlico County 2000 1.94782724762452970276 0.35933779785623271693 1.56217364489823888166 1.36537077933199287260 +Pawnee County 2000 1.78950195911489308028 1.16132303881758630677 3.42292675284956044317 0.55048211219399261479 +Pierce County 2000 2.54716933553443590360 1.41791053358023116142 1.22686753138250342480 1.16331445295301712751 +Pike County 2000 0.87756608639678817034 0.86487524757082952225 1.66450846493571420317 1.60752153195890666018 +Plymouth County 2000 0.42751024431730323179 0.24869467440858413355 1.77347026156010104138 0.95743334493327210261 +Pocahontas County 2000 1.81898673318791661987 1.68668175433656035469 2.78840493521175989923 2.31448886854618583141 +Pope County 2000 1.42325802767704214927 0.71289882499386886358 5.32400461162588232396 4.36612471915760924134 +Quitman County 2000 0.36883113368425186793 0.36314047030366149345 5.16370853507835506706 3.21226900707602706150 +Richland County 2000 3.25294641523656020681 1.07950401261765093476 2.63822689761623055166 1.41044850738739382893 +Rockland County 2000 0.42416114343500113082 0.20590865976918589352 3.77761891177163143690 3.03503154543555890597 +Salem city 2000 0.70139973333391227808 0.35839062570476603239 2.05906098316466157534 1.08354507133942051245 +Seminole County 2000 1.44053424523671633073 0.43123781775978624959 5.98509759915132505814 3.90425489309553190978 +Sherman County 2000 3.80549228460960365296 0.85018052505371507262 3.28429802752271011291 2.95060760395590827092 +Sioux County 2000 4.05794434246552571650 1.42665905155276116003 1.56803884104955881507 1.13419970938634791615 +Stafford County 2000 2.47320669600627229039 0.89946337873032839807 1.93845899918871104768 0.90136132522419968048 +Suffolk County 2000 2.42331649558212058212 1.82204858730891230166 6.53566355839665770211 1.95538998836281647012 +Sully County 2000 0.93205458749496189963 0.25090187329447210579 2.29778378271979174484 1.08802927271862880423 +Thomas County 2000 2.83931404059757943504 1.88036403728663893895 1.98634825955250608974 0.52141190626809911920 +Todd County 2000 2.82280525212475972969 0.89465756573513943151 3.16893938472753630323 1.86121311814899148488 +Valdez-Cordova Census Area 2000 1.25070886925917779038 0.59811263426251699657 1.22504083956972939990 0.84771422379395948262 +Waushara County 2000 7.21438509071107960599 3.30949368270859706271 0.60525565982701327843 0.60022615175317538544 +Wayne County 2000 0.94936213717458544629 0.78911846142834762231 2.95594340748058137863 1.96105182147450904414 +Williamson County 2000 6.40051753949925199369 0.73796090630650528678 2.74186195207121934591 2.27659084754941781749 +Woodford County 2000 1.66114001659859092820 0.55664623460469568767 1.82209706086687377022 1.17870749798601085737 +Wyoming County 2000 0.71452417457513699221 0.46814038549181972360 9.85342118017399816692 3.19644465036087064672 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q32.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q32.sql.out new file mode 100644 index 0000000000000..de81b93b81cc1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q32.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q33.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q33.sql.out new file mode 100644 index 0000000000000..668a1c44a6907 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q33.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +796 319.61 +820 445.38 +945 550.19 +753 655.92 +455 711.84 +564 739.49 +951 850.52 +776 1039.28 +676 1386.92 +911 1536.42 +832 1643.04 +925 1744.74 +701 1859.69 +707 1912.37 +780 2339.10 +891 2412.45 +890 3020.90 +870 3142.82 +656 3227.47 +681 3427.68 +985 3441.42 +881 3677.27 +717 3733.60 +854 3734.59 +449 4015.00 +965 4331.20 +960 4344.29 +621 4411.86 +756 4863.42 +918 5140.55 +795 5145.78 +833 5186.15 +976 5317.66 +696 5726.05 +853 5799.18 +770 5958.32 +722 6049.15 +362 6348.46 +399 6401.19 +973 6606.87 +715 6673.32 +708 6760.71 +537 7094.17 +724 7333.62 +375 7385.50 +444 7791.92 +716 7793.63 +887 7844.78 +464 7874.66 +709 7898.12 +978 7952.34 +893 8530.99 +677 8770.76 +595 8787.96 +869 9002.04 +338 9032.32 +738 9136.32 +429 9177.98 +650 9182.94 +901 9212.11 +921 9223.79 +946 9353.49 +858 9443.98 +905 9494.47 +568 9614.09 +389 9939.44 +836 9951.19 +821 9963.08 +728 10017.99 +679 10066.80 +721 10348.99 +910 10453.73 +698 10543.49 +777 10545.94 +675 10556.87 +783 10559.82 +624 10836.32 +763 10885.70 +730 10897.97 +463 10912.55 +534 10925.55 +913 10942.78 +970 11004.86 +846 11218.39 +742 11301.99 +794 11444.72 +442 11482.50 +393 11644.19 +706 11654.35 +577 11722.62 +929 11744.05 +733 11781.60 +896 11875.86 +807 11893.86 +386 12070.02 +865 12380.13 +561 12458.46 +552 12588.43 +683 12592.26 +997 12921.36 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q34.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q34.sql.out new file mode 100644 index 0000000000000..71b7844e8e93a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q34.sql.out @@ -0,0 +1,223 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL Y 47915 15 +NULL NULL NULL NULL 126143 15 +NULL NULL NULL NULL 215293 15 +NULL NULL Mrs. N 120593 15 +NULL Rubin Sir NULL 30056 15 +Adler Justin Sir Y 226187 15 +Allen Rose Mrs. N 179476 16 +Anderson Marvin Dr. N 211012 16 +Andrews Jacob Mr. N 67111 16 +Andrews Samuel Dr. Y 139993 16 +Angel Kevin Mr. Y 106628 15 +Ashley Linda Mrs. Y 82173 15 +Baca Dorothy Mrs. N 64890 15 +Baker Jamie Dr. Y 9916 15 +Banks Leroy Sir N 206730 15 +Barber Dianna Mrs. Y 119959 16 +Barksdale Joann Miss Y 138994 15 +Barnes Ruth Dr. N 84038 15 +Barney Samuel Sir N 15288 15 +Barnhart Charley Mr. Y 166576 15 +Barone Seth Mr. Y 162374 15 +Barrett David Sir N 189879 15 +Bartels Elmer Sir Y 114760 16 +Bear Scott Sir Y 82291 15 +Beers Kendra Dr. NULL 137960 15 +Belcher James Sir Y 239470 16 +Bell Carrie Miss N 5527 15 +Bell Matthew Dr. N 20400 15 +Benjamin Consuelo Ms. Y 201086 15 +Bergman Joann Miss N 177052 15 +Brooks Robert Sir N 155576 16 +Byrd Kelly Sir N 165115 16 +Cagle Jennifer Miss N 163129 15 +Campbell Robert Mr. N 8964 15 +Cardona Robert Mr. N 200501 15 +Carter Wendy Ms. N 96439 15 +Carver Bernard Mr. Y 194943 16 +Chen Wanita Miss N 137713 16 +Christensen Larry Dr. Y 58094 15 +Cochrane Anne Mrs. N 208347 16 +Coleman Inez Dr. Y 88249 16 +Coleman John Mr. N 49444 15 +Colon Anna Dr. Y 143694 15 +Conley Roxie Dr. N 196663 15 +Cook Adam Ms. Y 167339 15 +Cote Justin Dr. N 93466 15 +Council Donald Sir Y 102958 15 +Cramer Linda Ms. N 126628 15 +Crittenden Amie Ms. N 138787 15 +Cruz James Mr. Y 201430 15 +Cuellar Oscar Mr. Y 86781 16 +Cullen Larry Mr. Y 221242 16 +Cushing Antonia Mrs. Y 118927 15 +Davis Gordon Dr. N 227822 15 +Davis Myrtle Dr. Y 37430 15 +Decker Vera Miss Y 75737 16 +Diamond Fernando Dr. N 216391 15 +Diaz Walton Mr. N 131135 16 +Dickinson Steven Mr. N 8057 16 +Douglas Lester Sir N 26043 15 +Dove Garry Dr. N 152171 16 +Drake Rosetta Dr. Y 238040 15 +Dumas Travis Mr. Y 94154 15 +Duncan Olivia Dr. Y 102032 15 +Durham Andrea Dr. Y 144734 15 +Dutton Gay Miss Y 110886 15 +Ellis Karen Miss N 229706 16 +Ely Cesar Dr. Y 36054 16 +Etheridge Mike Dr. N 19648 15 +Farmer Eugenia Miss Y 98187 16 +Farrow Kathy Miss Y 200078 15 +Faulkner Lakeisha Dr. Y 178393 16 +Faulkner Robert Dr. N 109423 15 +Felton David Mr. N 97807 16 +Ferreira Christine Mrs. Y 155822 15 +Finn Robert Mr. N 38057 15 +Finney Crystal Miss Y 158304 15 +Fischer Tamara Mrs. N 66790 15 +Foote Roy Sir N 68086 15 +Foreman Autumn Mrs. Y 164060 15 +Funk Marvin Sir Y 61516 15 +Garcia Christopher Sir Y 181616 16 +Garcia Karen Miss N 236987 15 +Garcia Robert Dr. N 172185 16 +Garland Michael Mr. N 234421 15 +Gaylord Keith Mr. Y 123333 16 +Gifford Mark Mr. N 225973 16 +Gilbert NULL Sir N 16844 15 +Gilmore Austin Dr. Y 239871 15 +Goldsmith Bernice Ms. Y 2347 15 +Good Nancy Dr. N 132655 15 +Goodman NULL NULL N 71903 15 +Gower Nettie Miss N 10576 15 +Gray Evelyn Miss N 157486 15 +Hammond Roger Sir Y 54884 16 +Hardin Kimberly Dr. N 192424 16 +Harp Vance Mr. N 199017 15 +Harper Madeline Dr. N 173835 16 +Harris Tammy Dr. N 217761 16 +Hartmann Joey Dr. N 230915 15 +Hayes David Sir N 82274 15 +Haynes Sara Miss Y 139168 16 +Heath Matthew Dr. N 30710 15 +Hennessey Debbie Dr. Y 79256 15 +Herman Stella Ms. Y 33801 16 +Hernandez Max Mr. N 16015 15 +Hernandez Ruth Ms. Y 97000 15 +Hess Joseph Sir N 151336 15 +Hodges Lucas Dr. Y 163325 15 +Holland Jeremiah Dr. N 95938 16 +Jackson William Mr. Y 16425 16 +Jameson Miguel Dr. N 9181 16 +Jarrell Thomas Mr. Y 85787 16 +Johnson Julia Dr. N 27560 15 +Jones Theresa Ms. N 219765 16 +Kelly Mark Mr. Y 17039 16 +Khan Hank Mr. N 177803 15 +Kim Charlotte Dr. Y 7208 16 +Kunz Sarah Dr. N 74568 15 +Lake Robert Sir N 13264 15 +Landry Rudolph Sir N 117581 15 +Lane Luis Sir N 232302 16 +Langford Darlene Mrs. N 214891 15 +Larson Kevin Mr. Y 35053 15 +Larson Thomas Mr. N 114265 15 +Lee Malik Dr. N 20122 16 +Leonard Orlando Dr. Y 133168 15 +Lincoln Anthony Miss Y 1407 16 +Lindsey Linda Dr. N 62687 16 +Lopez Karen Dr. Y 136008 15 +Lunsford Kevin Dr. N 159120 16 +Lynch Sylvia Ms. Y 115438 15 +Lyon Michael Mr. N 140323 15 +Maestas Mabel Mrs. N 184265 15 +Magana Diann Mrs. Y 19139 15 +Manning Annamarie Ms. N 4984 16 +Marshall Felipe Sir N 138890 15 +Martin Paul Dr. N 26115 16 +Martinez Earl Sir N 108982 15 +Martinez Robert Sir Y 157672 16 +Masterson Barbara Mrs. N 231070 15 +Mata Deborah Miss Y 4323 15 +Mccoy Debbie Dr. N 91552 15 +Mcgill Tony Sir N 110030 15 +Mckeon Christina Dr. N 26190 15 +Mcnamara Linda Dr. Y 7957 15 +Means Michael Mr. Y 226164 16 +Medina Joseph Sir Y 110246 15 +Meyers Zachary Mr. Y 59549 15 +Montgomery John Mr. Y 103718 15 +Moody Miranda Ms. Y 171671 15 +Moore Mark Dr. N 191471 15 +Moran Celia Ms. Y 200691 15 +Morgan Cecelia Mrs. N 200742 15 +Morrell Chad Mr. N 93790 15 +Morse Robert Mr. N 68627 16 +Neel Audrey Ms. Y 193308 15 +Neff Sheri Mrs. Y 52556 15 +Nelson Katherine Mrs. N 110232 15 +New Suzanne Miss N 5120 16 +Nielsen Veronica Mrs. N 23905 15 +Oakley George Mr. Y 177890 15 +Parker Barbar Dr. N 57241 15 +Parker Jeff Sir N 213566 16 +Pemberton Jennifer Mrs. Y 49875 16 +Perry Robert Mr. Y 153147 16 +Phillips David Dr. N 148883 15 +Phillips Georgia NULL Y 26878 15 +Phillips Stanley Sir N 31989 15 +Pinkston Brenda Dr. N 126440 15 +Pryor Dorothy Mrs. N 213779 16 +Reed William Dr. N 145002 15 +Reynolds Amelia Ms. Y 68440 15 +Rice David Dr. Y 70484 16 +Robertson Daniel Mr. N 40407 16 +Rosales NULL NULL Y 156406 16 +Russo Cheryl Miss N 81123 15 +Sanchez Bruce Sir Y 124479 15 +Schmitz Kaitlyn Miss N 105162 15 +Sebastian Homer Dr. Y 64994 15 +Sexton Jerry Sir N 91446 15 +Sierra David Sir Y 61810 15 +Simmons Joseph Dr. N 54185 15 +Simpson Michael Sir Y 186613 16 +Simpson Shalanda Dr. Y 181123 15 +Singleton Andrew Ms. N 45464 15 +Smith Danny Dr. Y 143297 15 +Smith Edward Sir Y 81178 16 +Smith Hung Sir N 44710 15 +Smith Kimberly Mrs. Y 174638 15 +Smith Vern Sir N 50960 15 +Sosa Leah Ms. Y 77106 16 +Sparks Erick Dr. N 220337 15 +Taylor Kenneth Dr. Y 194337 15 +Todd Linda Ms. Y 235816 15 +Trout Harley Mr. Y 214547 15 +Urban NULL NULL NULL 214898 15 +Varner Elsie Ms. N 199602 16 +Vazquez Bill Dr. Y 62049 15 +Velazquez William Dr. N 46239 15 +Wagner Barbara Ms. Y 233595 15 +Ward Anna Miss N 52941 16 +Watkins Rosa Miss Y 152190 16 +Welch James Dr. Y 51441 16 +West Teresa Ms. N 233179 16 +White Maurice Mr. N 10107 15 +Williams John Mr. Y 84783 15 +Williams Robert Mr. Y 41233 15 +Williamson Ruth Mrs. Y 86369 15 +Wilson Joel Sir Y 91826 16 +Wilson John Sir Y 26543 15 +Wilson Mariano Mr. Y 67472 16 +Winkler Jose Dr. Y 78400 15 +Winter Cora Mrs. N 8978 16 +Wood Marcia Ms. Y 219276 16 +Wood Michelle Mrs. N 39560 15 +Wright Richie Sir Y 106818 15 +Young William Mr. Y 51127 15 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q35.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q35.sql.out new file mode 100644 index 0000000000000..3307163872566 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q35.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL F D 1 0 0 0.0 2 1 2 2 2.0 2 1 2 2 2.0 +NULL F D 1 0 0 0.0 3 1 3 3 3.0 4 1 4 4 4.0 +NULL F D 1 0 0 0.0 5 1 5 5 5.0 2 1 2 2 2.0 +NULL F D 1 0 0 0.0 6 1 6 6 6.0 4 1 4 4 4.0 +NULL F D 1 1 1 1.0 1 1 1 1 1.0 1 1 1 1 1.0 +NULL F D 1 1 1 1.0 4 1 4 4 4.0 4 1 4 4 4.0 +NULL F D 1 1 1 1.0 4 1 4 4 4.0 5 1 5 5 5.0 +NULL F D 1 2 2 2.0 0 1 0 0 0.0 4 1 4 4 4.0 +NULL F D 1 2 2 2.0 1 1 1 1 1.0 3 1 3 3 3.0 +NULL F D 1 2 2 2.0 6 1 6 6 6.0 1 1 1 1 1.0 +NULL F D 1 3 3 3.0 3 1 3 3 3.0 2 1 2 2 2.0 +NULL F D 1 3 3 3.0 3 1 3 3 3.0 6 1 6 6 6.0 +NULL F D 1 3 3 3.0 4 1 4 4 4.0 1 1 1 1 1.0 +NULL F D 1 4 4 4.0 0 1 0 0 0.0 3 1 3 3 3.0 +NULL F D 1 4 4 4.0 1 1 1 1 1.0 1 1 1 1 1.0 +NULL F D 1 4 4 4.0 1 1 1 1 1.0 4 1 4 4 4.0 +NULL F D 1 4 4 4.0 5 1 5 5 5.0 6 1 6 6 6.0 +NULL F D 1 5 5 5.0 4 1 4 4 4.0 3 1 3 3 3.0 +NULL F D 1 5 5 5.0 5 1 5 5 5.0 2 1 2 2 2.0 +NULL F D 1 6 6 6.0 1 1 1 1 1.0 3 1 3 3 3.0 +NULL F D 1 6 6 6.0 2 1 2 2 2.0 2 1 2 2 2.0 +NULL F D 1 6 6 6.0 4 1 4 4 4.0 1 1 1 1 1.0 +NULL F M 1 0 0 0.0 5 1 5 5 5.0 5 1 5 5 5.0 +NULL F M 1 1 1 1.0 3 1 3 3 3.0 0 1 0 0 0.0 +NULL F M 1 1 1 1.0 6 1 6 6 6.0 0 1 0 0 0.0 +NULL F M 1 1 1 1.0 6 1 6 6 6.0 1 1 1 1 1.0 +NULL F M 1 2 2 2.0 2 1 2 2 2.0 6 1 6 6 6.0 +NULL F M 1 2 2 2.0 4 1 4 4 4.0 4 1 4 4 4.0 +NULL F M 1 3 3 3.0 2 1 2 2 2.0 1 1 1 1 1.0 +NULL F M 1 3 3 3.0 5 1 5 5 5.0 0 1 0 0 0.0 +NULL F M 1 3 3 3.0 5 1 5 5 5.0 1 1 1 1 1.0 +NULL F M 1 4 4 4.0 1 1 1 1 1.0 4 1 4 4 4.0 +NULL F M 1 4 4 4.0 2 1 2 2 2.0 1 1 1 1 1.0 +NULL F M 1 4 4 4.0 3 1 3 3 3.0 3 1 3 3 3.0 +NULL F M 1 5 5 5.0 2 1 2 2 2.0 2 1 2 2 2.0 +NULL F M 1 6 6 6.0 1 1 1 1 1.0 1 1 1 1 1.0 +NULL F M 1 6 6 6.0 5 1 5 5 5.0 6 1 6 6 6.0 +NULL F S 1 0 0 0.0 3 1 3 3 3.0 6 1 6 6 6.0 +NULL F S 1 1 1 1.0 0 1 0 0 0.0 4 1 4 4 4.0 +NULL F S 1 1 1 1.0 1 1 1 1 1.0 2 1 2 2 2.0 +NULL F S 1 1 1 1.0 2 1 2 2 2.0 6 1 6 6 6.0 +NULL F S 1 1 1 1.0 5 1 5 5 5.0 5 1 5 5 5.0 +NULL F S 1 2 2 2.0 0 1 0 0 0.0 3 1 3 3 3.0 +NULL F S 2 2 2 2.0 5 2 5 5 5.0 6 2 6 6 6.0 +NULL F S 1 3 3 3.0 0 1 0 0 0.0 4 1 4 4 4.0 +NULL F S 1 3 3 3.0 2 1 2 2 2.0 1 1 1 1 1.0 +NULL F S 1 3 3 3.0 2 1 2 2 2.0 5 1 5 5 5.0 +NULL F S 1 3 3 3.0 3 1 3 3 3.0 3 1 3 3 3.0 +NULL F S 1 4 4 4.0 1 1 1 1 1.0 4 1 4 4 4.0 +NULL F S 1 4 4 4.0 2 1 2 2 2.0 4 1 4 4 4.0 +NULL F S 1 5 5 5.0 6 1 6 6 6.0 0 1 0 0 0.0 +NULL F U 1 0 0 0.0 1 1 1 1 1.0 3 1 3 3 3.0 +NULL F U 1 0 0 0.0 3 1 3 3 3.0 0 1 0 0 0.0 +NULL F U 1 1 1 1.0 3 1 3 3 3.0 2 1 2 2 2.0 +NULL F U 1 1 1 1.0 5 1 5 5 5.0 6 1 6 6 6.0 +NULL F U 1 2 2 2.0 0 1 0 0 0.0 1 1 1 1 1.0 +NULL F U 1 2 2 2.0 4 1 4 4 4.0 4 1 4 4 4.0 +NULL F U 2 3 3 3.0 1 2 1 1 1.0 6 2 6 6 6.0 +NULL F U 1 4 4 4.0 0 1 0 0 0.0 4 1 4 4 4.0 +NULL F U 1 5 5 5.0 3 1 3 3 3.0 6 1 6 6 6.0 +NULL F U 1 6 6 6.0 2 1 2 2 2.0 2 1 2 2 2.0 +NULL F U 1 6 6 6.0 4 1 4 4 4.0 4 1 4 4 4.0 +NULL F U 1 6 6 6.0 5 1 5 5 5.0 0 1 0 0 0.0 +NULL F U 1 6 6 6.0 5 1 5 5 5.0 6 1 6 6 6.0 +NULL F W 1 0 0 0.0 0 1 0 0 0.0 4 1 4 4 4.0 +NULL F W 1 0 0 0.0 5 1 5 5 5.0 5 1 5 5 5.0 +NULL F W 1 1 1 1.0 3 1 3 3 3.0 4 1 4 4 4.0 +NULL F W 1 2 2 2.0 0 1 0 0 0.0 5 1 5 5 5.0 +NULL F W 1 3 3 3.0 3 1 3 3 3.0 6 1 6 6 6.0 +NULL F W 1 3 3 3.0 6 1 6 6 6.0 6 1 6 6 6.0 +NULL F W 1 4 4 4.0 3 1 3 3 3.0 1 1 1 1 1.0 +NULL F W 1 5 5 5.0 1 1 1 1 1.0 1 1 1 1 1.0 +NULL F W 1 5 5 5.0 1 1 1 1 1.0 4 1 4 4 4.0 +NULL F W 1 5 5 5.0 3 1 3 3 3.0 6 1 6 6 6.0 +NULL F W 1 5 5 5.0 4 1 4 4 4.0 6 1 6 6 6.0 +NULL F W 1 6 6 6.0 0 1 0 0 0.0 5 1 5 5 5.0 +NULL F W 1 6 6 6.0 2 1 2 2 2.0 3 1 3 3 3.0 +NULL F W 1 6 6 6.0 5 1 5 5 5.0 5 1 5 5 5.0 +NULL M D 1 0 0 0.0 3 1 3 3 3.0 0 1 0 0 0.0 +NULL M D 1 1 1 1.0 3 1 3 3 3.0 0 1 0 0 0.0 +NULL M D 1 1 1 1.0 3 1 3 3 3.0 2 1 2 2 2.0 +NULL M D 1 2 2 2.0 0 1 0 0 0.0 6 1 6 6 6.0 +NULL M D 1 2 2 2.0 4 1 4 4 4.0 4 1 4 4 4.0 +NULL M D 1 2 2 2.0 5 1 5 5 5.0 3 1 3 3 3.0 +NULL M D 1 3 3 3.0 1 1 1 1 1.0 5 1 5 5 5.0 +NULL M D 1 3 3 3.0 2 1 2 2 2.0 3 1 3 3 3.0 +NULL M D 1 4 4 4.0 5 1 5 5 5.0 2 1 2 2 2.0 +NULL M D 1 6 6 6.0 1 1 1 1 1.0 6 1 6 6 6.0 +NULL M D 1 6 6 6.0 3 1 3 3 3.0 1 1 1 1 1.0 +NULL M M 1 0 0 0.0 0 1 0 0 0.0 1 1 1 1 1.0 +NULL M M 2 0 0 0.0 1 2 1 1 1.0 2 2 2 2 2.0 +NULL M M 1 0 0 0.0 2 1 2 2 2.0 1 1 1 1 1.0 +NULL M M 1 0 0 0.0 3 1 3 3 3.0 5 1 5 5 5.0 +NULL M M 1 0 0 0.0 5 1 5 5 5.0 0 1 0 0 0.0 +NULL M M 1 1 1 1.0 0 1 0 0 0.0 1 1 1 1 1.0 +NULL M M 1 1 1 1.0 0 1 0 0 0.0 2 1 2 2 2.0 +NULL M M 1 2 2 2.0 6 1 6 6 6.0 5 1 5 5 5.0 +NULL M M 1 3 3 3.0 5 1 5 5 5.0 1 1 1 1 1.0 +NULL M M 1 3 3 3.0 6 1 6 6 6.0 4 1 4 4 4.0 +NULL M M 1 4 4 4.0 1 1 1 1 1.0 3 1 3 3 3.0 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q36.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q36.sql.out new file mode 100644 index 0000000000000..9fd97cbed90ed --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q36.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +-0.43310777864678831165 NULL NULL 2 1 +-0.44057752675240550259 Home NULL 1 1 +-0.43759152110176221048 Music NULL 1 2 +-0.43708103961494058652 NULL NULL 1 3 +-0.43616253138693450880 Shoes NULL 1 4 +-0.43567118609322457134 Children NULL 1 5 +-0.43423932351647837678 Sports NULL 1 6 +-0.43342977299642408093 Electronics NULL 1 7 +-0.43243283120699560700 Women NULL 1 8 +-0.43164166899823508408 Men NULL 1 9 +-0.42516187689954540402 Books NULL 1 10 +-0.42448713380832790884 Jewelry NULL 1 11 +-0.73902664238792748962 NULL shirts 0 1 +-0.61125804873635587486 NULL country 0 2 +-0.53129803597069255822 NULL dresses 0 3 +-0.51266635289382758517 NULL athletic 0 4 +-0.45290387783638603924 NULL mens 0 5 +-0.41288056661656330013 NULL accessories 0 6 +-0.40784754677005682440 NULL NULL 0 7 +-0.34254844860867375832 NULL baseball 0 8 +-0.32511461675631534897 NULL infants 0 9 +-0.44733955704648003493 Books computers 0 1 +-0.44221358112622373783 Books home repair 0 2 +-0.44131129175272951442 Books romance 0 3 +-0.43954111564375046074 Books history 0 4 +-0.43921337505389731821 Books mystery 0 5 +-0.43904020269360481109 Books sports 0 6 +-0.42821476999837619396 Books travel 0 7 +-0.42609067296303848297 Books cooking 0 8 +-0.42538995145338568328 Books fiction 0 9 +-0.42446563616188232944 Books arts 0 10 +-0.42424821311884350413 Books parenting 0 11 +-0.41822014479424203008 Books reference 0 12 +-0.41350839325516811781 Books business 0 13 +-0.40935208137315013129 Books science 0 14 +-0.40159380735731858928 Books self-help 0 15 +-0.36957884843305744526 Books entertainments 0 16 +-0.44602461556731552282 Children school-uniforms 0 1 +-0.44141106040000560852 Children toddlers 0 2 +-0.43479886701046623711 Children infants 0 3 +-0.41900662971936329442 Children newborn 0 4 +-0.41526603781609697786 Children NULL 0 5 +-0.45347482218635333366 Electronics personal 0 1 +-0.44349670349829474271 Electronics stereo 0 2 +-0.44262427232850112058 Electronics automotive 0 3 +-0.44115886172705231970 Electronics portable 0 4 +-0.43972786651639318010 Electronics memory 0 5 +-0.43889275271590953040 Electronics scanners 0 6 +-0.43879181695132886061 Electronics karoke 0 7 +-0.43743655149948399284 Electronics dvd/vcr players 0 8 +-0.43737666390514154910 Electronics cameras 0 9 +-0.43390499017233926812 Electronics wireless 0 10 +-0.43163869754114299547 Electronics audio 0 11 +-0.42958938669780912634 Electronics camcorders 0 12 +-0.42872845803629855724 Electronics musical 0 13 +-0.42228240153396399656 Electronics televisions 0 14 +-0.41893847772039275795 Electronics monitors 0 15 +-0.39793878022746331540 Electronics disk drives 0 16 +-0.49051156860507320113 Home NULL 0 1 +-0.48431476750686752965 Home blinds/shades 0 2 +-0.47545837941951440918 Home bathroom 0 3 +-0.45726228921216284093 Home rugs 0 4 +-0.45540507568891021759 Home furniture 0 5 +-0.45303572267019508501 Home flatware 0 6 +-0.44755542058111800358 Home tables 0 7 +-0.44419847780930149402 Home wallpaper 0 8 +-0.44092345226680695671 Home glassware 0 9 +-0.43877591834074789745 Home decor 0 10 +-0.43765482553654514822 Home accent 0 11 +-0.43188199218974854630 Home bedding 0 12 +-0.43107417904272222899 Home kids 0 13 +-0.42474436355625900935 Home lighting 0 14 +-0.41783311109052416746 Home curtains/drapes 0 15 +-0.41767111806961188479 Home mattresses 0 16 +-0.40562188698541221499 Home paint 0 17 +-0.45165056505480816921 Jewelry jewelry boxes 0 1 +-0.44372227804836590137 Jewelry estate 0 2 +-0.44251815032563188894 Jewelry gold 0 3 +-0.43978127753996883542 Jewelry consignment 0 4 +-0.43821750044359339153 Jewelry custom 0 5 +-0.43439645036479672989 Jewelry bracelets 0 6 +-0.43208398325687772942 Jewelry loose stones 0 7 +-0.43060897375114375156 Jewelry diamonds 0 8 +-0.42847505748860847066 Jewelry costume 0 9 +-0.42667449062277843561 Jewelry rings 0 10 +-0.41987969011585456826 Jewelry mens watch 0 11 +-0.41624621972944533035 Jewelry semi-precious 0 12 +-0.41148949162100715771 Jewelry womens watch 0 13 +-0.39725668174847694299 Jewelry birdal 0 14 +-0.39665274051903254057 Jewelry pendants 0 15 +-0.38423525233438861010 Jewelry earings 0 16 +-0.44464388887858793403 Men shirts 0 1 +-0.43719860800637369827 Men accessories 0 2 +-0.43164606665359630905 Men sports-apparel 0 3 +-0.41530906677293519754 Men pants 0 4 +-0.38332708894803499123 Men NULL 0 5 +-0.47339698705534020269 Music NULL 0 1 +-0.44193214675249008923 Music rock 0 2 +-0.44008174913565459246 Music country 0 3 +-0.43863444992223641373 Music pop 0 4 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q37.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q37.sql.out new file mode 100644 index 0000000000000..1782f9025fa1e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q37.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAADHGDAAAA Necessary times believe probably. Cruel traders know ho 92.95 +AAAAAAAAFMLDAAAA Given groups please unfortu 84.79 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q38.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q38.sql.out new file mode 100644 index 0000000000000..7bcc92dc20802 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q38.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +104 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q39a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q39a.sql.out new file mode 100644 index 0000000000000..f2b8dc07af08d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q39a.sql.out @@ -0,0 +1,211 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1 823 1 301.5 1.1271370519097714 1 823 2 399.25 1.0208768007427147 +1 1015 1 344.25 1.2511428113709673 1 1015 2 385.5 1.0470314051933909 +1 1555 1 264.25 1.2283186159550554 1 1555 2 308.25 1.058767914170086 +1 1691 1 116.33333333333333 1.0248470977646387 1 1691 2 304.75 1.1191291694661885 +1 1859 1 434.0 1.0498116619056204 1 1859 2 314.0 1.1900690532464868 +1 2239 1 322.5 1.0978977568088866 1 2239 2 382.75 1.0474268515036576 +1 3859 1 352.25 1.1782246058681007 1 3859 2 437.5 1.145917046540028 +1 4975 1 388.5 1.0001018617093298 1 4975 2 363.3333333333333 1.0038038922115327 +1 5109 1 370.75 1.1395542642353356 1 5109 2 322.25 1.1495609463470746 +1 5177 1 292.0 1.3461430973363442 1 5177 2 216.25 1.05761161489175 +1 6619 1 409.5 1.0040570532236324 1 6619 2 322.3333333333333 1.0619035677903699 +1 7535 1 428.25 1.0544682695099616 1 7535 2 391.5 1.222946431626478 +1 8283 1 194.25 1.2003971691624762 1 8283 2 400.6666666666667 1.1209666759436547 +1 8401 1 328.25 1.0063843177755347 1 8401 2 297.75 1.4183634329309731 +1 8547 1 328.25 1.1425055398027557 1 8547 2 264.5 1.0416250678095451 +1 8717 1 228.5 1.2085987127693683 1 8717 2 288.25 1.2149351569032296 +1 8933 1 361.0 1.1736874031675129 1 8933 2 433.0 1.0089048763341881 +1 8997 1 250.0 1.035266149354841 1 8997 2 430.6666666666667 1.1313205669952624 +1 9245 1 392.3333333333333 1.1119320768290193 1 9245 2 343.0 1.1904615139612167 +1 9621 1 277.6666666666667 1.2243644452469618 1 9621 2 337.25 1.2482331965937552 +1 10299 1 380.5 1.1949268339105126 1 10299 2 144.5 1.184591012875645 +1 10745 1 287.3333333333333 1.084129539111935 1 10745 2 335.6666666666667 1.1651939735017574 +1 11125 1 472.75 1.0530701492045889 1 11125 2 264.5 1.0730737223380644 +1 11859 1 256.5 1.4406552725835113 1 11859 2 370.75 1.1082243282754298 +1 12101 1 334.75 1.1423000747353222 1 12101 2 396.5 1.0099457859523537 +1 12259 1 326.5 1.219693210219279 1 12259 2 292.6666666666667 1.2808898286830026 +1 12641 1 321.25 1.1286221893301993 1 12641 2 279.25 1.129134558577743 +1 13043 1 260.5 1.355894484625015 1 13043 2 295.0 1.056210118409035 +1 13157 1 260.5 1.5242630430075292 1 13157 2 413.5 1.0422561797285326 +1 13293 1 325.25 1.1599721810918615 1 13293 2 345.75 1.0626233629994524 +1 13729 1 486.0 1.0680776434770018 1 13729 2 389.6666666666667 1.3522269473359647 +1 14137 1 427.0 1.0418229612154228 1 14137 2 387.5 1.0294855239302605 +1 14159 1 398.6666666666667 1.0001328196713188 1 14159 2 186.5 1.01269532733355 +1 14161 1 84.66666666666667 1.4291501268026987 1 14161 2 450.5 1.0059037693687187 +1 14911 1 397.6666666666667 1.1801953152598252 1 14911 2 195.0 1.0294118715204632 +1 14983 1 286.6666666666667 1.335077674103522 1 14983 2 138.25 1.0223829110804588 +1 15181 1 270.0 1.0247553155705627 1 15181 2 431.25 1.0014337967301454 +1 15425 1 311.5 1.0406385606636968 1 15425 2 348.25 1.1725274574998379 +1 15441 1 295.0 1.597573089984185 1 15441 2 296.0 1.534664092337063 +1 16601 1 428.5 1.0395250642903893 1 16601 2 397.75 1.108690797111687 +1 16895 1 197.25 1.6216484458744376 1 16895 2 201.33333333333334 1.001838963026182 +1 17285 1 291.5 1.221338355118313 1 17285 2 384.3333333333333 1.3166511804104957 +1 17399 1 335.25 1.18017157377048 1 17399 2 208.0 1.2199892235676928 +1 17521 1 248.0 1.1446287380504054 1 17521 2 321.25 1.1065301391790767 +2 39 1 306.75 1.3140265341053214 2 39 2 207.75 1.0612481315699458 +2 575 1 139.75 1.0651946548671536 2 575 2 390.0 1.2985907498394962 +2 577 1 398.3333333333333 1.0782049277126313 2 577 2 125.25 1.015309703853557 +2 795 1 445.0 1.008215213597613 2 795 2 226.0 1.0374761691860421 +2 977 1 383.0 1.0133755425339792 2 977 2 365.25 1.013000783116057 +2 993 1 361.0 1.0341824955539196 2 993 2 228.0 1.2304019216861168 +2 1367 1 402.0 1.143084675277277 2 1367 2 204.25 1.0427544150441603 +2 2863 1 366.5 1.0309651709854288 2 2863 2 216.75 1.213127273469588 +2 3041 1 330.75 1.0432879984065397 2 3041 2 397.25 1.0315273164376213 +2 3323 1 427.25 1.0090121310722835 2 3323 2 338.5 1.0459307196329675 +2 3999 1 422.0 1.0082188847601554 2 3999 2 403.25 1.0051474075339424 +2 4023 1 410.3333333333333 1.0275376199691826 2 4023 2 247.75 1.1609399487145424 +2 4331 1 384.75 1.064007972514113 2 4331 2 385.75 1.011321832432027 +2 5915 1 231.25 1.8705235468767831 2 5915 2 407.0 1.0885509926353207 +2 6275 1 420.6666666666667 1.008294829045718 2 6275 2 297.6666666666667 1.4615952762209015 +2 7087 1 494.5 1.0102112511298167 2 7087 2 148.5 1.7921780145107034 +2 7333 1 150.33333333333334 1.433050233356339 2 7333 2 214.75 1.0557216871980746 +2 7497 1 360.25 1.0793771564436658 2 7497 2 200.0 1.2799283834131763 +2 8825 1 381.5 1.097428368281076 2 8825 2 414.25 1.0153496363346763 +2 10795 1 393.3333333333333 1.0666650582315749 2 10795 2 322.75 1.0370108734399346 +2 10863 1 330.75 1.0627419399194518 2 10863 2 276.0 1.6689178243621974 +2 11073 1 274.0 1.6770433716259476 2 11073 2 359.0 1.0133649606830526 +2 11465 1 312.25 1.1867998857912492 2 11465 2 154.0 1.1105857906239676 +2 11853 1 292.25 1.0772136418065505 2 11853 2 401.5 1.0787107704829733 +2 12389 1 366.25 1.067334394007718 2 12389 2 278.5 1.1044576295014297 +2 13247 1 356.0 1.0660645893691494 2 13247 2 342.5 1.0634325916808434 +2 13661 1 293.0 1.2174529148923212 2 13661 2 412.25 1.048653193049242 +2 13923 1 335.75 1.2358541998052608 2 13923 2 100.75 1.0814570294681372 +2 14671 1 262.75 1.0250871002782607 2 14671 2 338.75 1.0628054504205149 +2 15397 1 297.6666666666667 1.2809438534554334 2 15397 2 492.75 1.0943752797356943 +2 15477 1 251.0 1.124353329693781 2 15477 2 307.6666666666667 1.3795297219800364 +2 15795 1 304.5 1.0607926164522463 2 15795 2 397.25 1.1176167493994051 +2 16603 1 293.3333333333333 1.387199822342635 2 16603 2 433.3333333333333 1.1660592106922516 +2 16969 1 364.6666666666667 1.1611696936141274 2 16969 2 375.3333333333333 1.0294034440006494 +2 17393 1 179.25 1.1070920857377156 2 17393 2 294.25 1.1481110921426008 +3 29 1 438.25 1.0131261466664097 3 29 2 344.0 1.1151530577310618 +3 247 1 321.0 1.0042014719826915 3 247 2 423.0 1.180182949214427 +3 953 1 338.75 1.1838346915880587 3 953 2 321.5 1.2363739805879619 +3 1541 1 110.0 1.1791578448793427 3 1541 2 273.75 1.197487924282276 +3 1649 1 360.6666666666667 1.1853733590339803 3 1649 2 334.25 1.1482623798952447 +3 2459 1 313.75 1.0048197369511642 3 2459 2 352.75 1.21947122536524 +3 2619 1 241.0 1.1159485992928209 3 2619 2 261.0 1.1099544779211474 +3 2707 1 375.25 1.1207806068743988 3 2707 2 290.75 1.0006820492941273 +3 2975 1 304.0 1.0591594463002163 3 2975 2 190.0 1.2046769431661426 +3 3315 1 271.75 1.555976998814345 3 3315 2 393.75 1.0196319345405949 +3 3393 1 260.0 1.5009563026568116 3 3393 2 470.25 1.129275872154205 +3 3597 1 304.0 1.2471400801439207 3 3597 2 364.0 1.057917059038131 +3 3661 1 331.25 1.2138186201312904 3 3661 2 398.25 1.0134502284121254 +3 3951 1 328.3333333333333 1.3920958631929026 3 3951 2 378.0 1.057830622993178 +3 4793 1 439.5 1.3208979917045633 3 4793 2 298.6666666666667 1.2536383791454593 +3 5221 1 395.25 1.012020609314844 3 5221 2 423.6666666666667 1.0742618083358388 +3 5857 1 331.5 1.1548423818657882 3 5857 2 394.3333333333333 1.101836576034495 +3 6045 1 313.5 1.1971443861134845 3 6045 2 67.25 1.2083633449201445 +3 6615 1 366.0 1.4103495908912012 3 6615 2 228.0 1.0322683130436006 +3 7071 1 182.75 1.402155194063468 3 7071 2 438.25 1.0176436798626307 +3 7211 1 355.25 1.2455338321801286 3 7211 2 462.0 1.0449517641148738 +3 8761 1 253.75 1.1207897246865177 3 8761 2 212.5 1.1557740307473354 +3 9305 1 350.6666666666667 1.3141448475357504 3 9305 2 387.3333333333333 1.043391324490137 +3 9373 1 179.75 1.3318949893741667 3 9373 2 321.25 1.1314604181366261 +3 9669 1 315.75 1.093783081996044 3 9669 2 321.0 1.1239703852823903 +3 9699 1 362.25 1.0269679854596525 3 9699 2 358.0 1.5025258842887776 +3 10301 1 348.5 1.2820855632941448 3 10301 2 318.5 1.289483896046129 +3 10427 1 241.33333333333334 1.5634035191786233 3 10427 2 381.25 1.0623056061004696 +3 11103 1 260.25 1.0537747255764836 3 11103 2 334.0 1.2702517027303248 +3 11141 1 251.0 1.0896833134701018 3 11141 2 272.0 1.1910327315841194 +3 12019 1 362.25 1.0966647561341047 3 12019 2 282.25 1.0983756663144604 +3 12743 1 276.5 1.005648259467935 3 12743 2 352.5 1.08876682930328 +3 12753 1 250.25 1.3386846981823803 3 12753 2 468.0 1.0383135087299893 +3 12931 1 322.75 1.1146291380437745 3 12931 2 320.0 1.17009448069376 +3 13487 1 308.25 1.2961991776642086 3 13487 2 293.25 1.0585525936033 +3 13555 1 373.25 1.0745070114317623 3 13555 2 152.75 1.412684197862033 +3 13581 1 292.75 1.1902035296028353 3 13581 2 253.0 1.155002663143799 +3 13829 1 233.5 1.1312399620732085 3 13829 2 444.25 1.0391188483200453 +3 13847 1 417.5 1.039525557170396 3 13847 2 263.5 1.4436108729741235 +3 14073 1 355.5 1.0476440697241391 3 14073 2 437.25 1.0172135605078851 +3 14767 1 311.5 1.0034195608338836 3 14767 2 339.0 1.2032144276415566 +3 14981 1 193.25 1.0060336654947306 3 14981 2 441.5 1.3661655364714043 +3 16331 1 272.0 1.1467170493846688 3 16331 2 339.25 1.2786638701439956 +3 16847 1 273.6666666666667 1.3346016934186173 3 16847 2 398.0 1.2041547394959626 +3 16987 1 358.0 1.101510614957325 3 16987 2 420.0 1.0848663494738469 +3 17613 1 424.3333333333333 1.0320925947787334 3 17613 2 390.75 1.0761214357356987 +3 17987 1 387.5 1.1128327233395303 3 17987 2 131.66666666666666 1.1227241574530091 +4 225 1 180.0 1.2847074573726138 4 225 2 366.5 1.112494070167504 +4 299 1 293.5 1.135267940218844 4 299 2 380.0 1.0485028679413595 +4 825 1 223.25 1.2488574716961685 4 825 2 254.0 1.5182802586094637 +4 1393 1 418.75 1.0408989038120988 4 1393 2 413.3333333333333 1.1020163503416796 +4 1523 1 363.25 1.0130673543588669 4 1523 2 253.5 1.2817761298828965 +4 1729 1 313.25 1.3148930771572687 4 1729 2 296.5 1.210664179669432 +4 2989 1 424.6666666666667 1.03767453099966 4 2989 2 123.75 1.4454541925191389 +4 3183 1 190.0 1.2520196057807818 4 3183 2 245.0 1.0300119488354766 +4 4175 1 395.0 1.042998032908585 4 4175 2 485.0 1.0145126110736231 +4 4293 1 285.0 1.042264740588342 4 4293 2 331.25 1.0702681575369872 +4 4573 1 243.75 1.4457774863358526 4 4573 2 431.25 1.0010829394909448 +4 4875 1 401.0 1.0066599946104444 4 4875 2 410.5 1.051550593497737 +4 5009 1 386.5 1.0301582587751055 4 5009 2 473.75 1.055073121585445 +4 5947 1 291.5 1.046282184237671 4 5947 2 320.5 1.1280002765664996 +4 6359 1 193.33333333333334 1.2483139639831744 4 6359 2 371.75 1.0993680760045068 +4 6517 1 289.0 1.0911931716633327 4 6517 2 148.0 1.0471156482980475 +4 8309 1 371.0 1.2845214196617782 4 8309 2 371.5 1.0748626938819539 +4 8339 1 392.75 1.0058445869354098 4 8339 2 345.75 1.2872431560206488 +4 9685 1 288.75 1.0017436994234579 4 9685 2 440.75 1.0083448738924952 +4 10255 1 373.5 1.1222827247788254 4 10255 2 352.0 1.1003307048901103 +4 10925 1 199.5 1.3875238422301213 4 10925 2 261.75 1.283642511996497 +4 11213 1 226.66666666666666 1.09984270658979 4 11213 2 413.5 1.0174813417315496 +4 11305 1 351.75 1.1922401157939606 4 11305 2 365.25 1.1258535465411879 +4 11473 1 394.6666666666667 1.0178948794541924 4 11473 2 212.66666666666666 1.195359710715888 +4 12353 1 340.25 1.164721531085477 4 12353 2 432.0 1.0523203480868901 +4 12783 1 329.5 1.0329266474827115 4 12783 2 187.0 1.2621302720196819 +4 12971 1 370.3333333333333 1.097620185659271 4 12971 2 278.0 1.4524982093215804 +4 13665 1 363.0 1.04089223995917 4 13665 2 332.6666666666667 1.1900176061910035 +4 13913 1 297.3333333333333 1.071040936419414 4 13913 2 316.25 1.3567449933143143 +4 15161 1 305.75 1.3571548565863678 4 15161 2 262.3333333333333 1.2292106140967536 +4 15401 1 341.0 1.0164918336889106 4 15401 2 337.25 1.0178529534898602 +4 15467 1 355.3333333333333 1.27670099607062 4 15467 2 416.6666666666667 1.1678517714162187 +4 16211 1 257.6666666666667 1.6381074811154002 4 16211 2 352.25 1.055236934125639 +4 16367 1 344.25 1.1865617643407205 4 16367 2 330.5 1.001436680208246 +4 16623 1 174.75 1.1547312605990323 4 16623 2 261.75 1.4692073123565808 +4 16753 1 283.6666666666667 1.4179905875607177 4 16753 2 331.0 1.0757450815976775 +4 16791 1 229.75 1.0415889892942085 4 16791 2 348.75 1.2365182061688882 +5 507 1 360.5 1.0016609878348282 5 507 2 397.25 1.1165805580468837 +5 1379 1 418.3333333333333 1.1593756930293735 5 1379 2 362.25 1.1381161323894302 +5 1451 1 259.75 1.0166115859746467 5 1451 2 186.5 1.3009837449067687 +5 1761 1 245.25 1.0674277258886877 5 1761 2 356.3333333333333 1.0319105846046546 +5 1919 1 558.0 1.051789656603646 5 1919 2 280.75 1.435982403616447 +5 2153 1 398.25 1.038369445511033 5 2153 2 322.0 1.2495167076207327 +5 2583 1 357.25 1.0689747703230787 5 2583 2 321.5 1.174109700061395 +5 2725 1 306.25 1.0685532393228003 5 2725 2 193.5 1.4095901314659105 +5 3547 1 357.0 1.1544864737016736 5 3547 2 343.75 1.2077817108886129 +5 3785 1 215.5 1.231057632809026 5 3785 2 460.0 1.048938011267006 +5 4445 1 327.3333333333333 1.0177488158574015 5 4445 2 414.75 1.046288264177383 +5 4601 1 327.25 1.15815714609041 5 4601 2 142.66666666666666 1.2197537262761011 +5 5019 1 341.6666666666667 1.2014886661438384 5 5019 2 363.5 1.1056740335885162 +5 5635 1 275.75 1.003161317494043 5 5635 2 195.33333333333334 1.49437494371756 +5 5725 1 308.6666666666667 1.2494665767967896 5 5725 2 315.5 1.4329959977644893 +5 5787 1 335.75 1.3581868619406905 5 5787 2 453.5 1.031825110180606 +5 8665 1 257.75 1.8119629759287612 5 8665 2 368.0 1.0243808356311228 +5 9037 1 189.75 1.0334701022027994 5 9037 2 326.0 1.1188906978754734 +5 9241 1 342.75 1.037524616861255 5 9241 2 174.5 1.1953290244295067 +5 9245 1 339.5 1.0092696575112496 5 9245 2 303.0 1.2214283206597227 +5 9789 1 391.25 1.0458503093728178 5 9789 2 343.75 1.070040394695916 +5 10775 1 439.5 1.2565424257262654 5 10775 2 330.75 1.3194508007529422 +5 10851 1 296.25 1.4450973535233087 5 10851 2 185.75 1.0920078306591938 +5 11409 1 337.5 1.2675445661798022 5 11409 2 267.5 1.2735562175240271 +5 11543 1 373.75 1.1069130009236565 5 11543 2 347.0 1.0384881272212296 +5 11907 1 312.0 1.1200627653353177 5 11907 2 130.0 1.0970913669588425 +5 12315 1 255.75 1.118436212304132 5 12315 2 329.25 1.1943065884369533 +5 12589 1 372.75 1.1109412437666168 5 12589 2 355.75 1.0853990131935114 +5 12853 1 306.75 1.4054585232279222 5 12853 2 174.75 1.0143495332647992 +5 13001 1 420.3333333333333 1.0350551797504086 5 13001 2 214.0 1.115488847819715 +5 13399 1 304.75 1.1656588906680398 5 13399 2 359.5 1.0278593838157082 +5 13809 1 338.0 1.3356560129512516 5 13809 2 254.25 1.3229081483155771 +5 13997 1 372.75 1.0067987273536196 5 13997 2 488.5 1.0136437168469898 +5 14683 1 313.75 1.0038487229336792 5 14683 2 344.5 1.1175817890990485 +5 14721 1 420.0 1.0064164891633403 5 14721 2 280.5 1.0511776533295065 +5 14891 1 445.25 1.0918502542585706 5 14891 2 420.3333333333333 1.1234936159879627 +5 15261 1 514.0 1.0718025790789742 5 15261 2 367.25 1.015824493979522 +5 15477 1 225.0 1.1250854562879289 5 15477 2 235.75 1.0790388329037188 +5 15655 1 367.5 1.100326204582237 5 15655 2 306.3333333333333 1.0511442359018315 +5 15673 1 351.25 1.0192896453224356 5 15673 2 389.6666666666667 1.0824061270803669 +5 15959 1 414.0 1.0773325961138016 5 15959 2 237.25 1.4024256583845858 +5 17339 1 399.25 1.0240754930004161 5 17339 2 265.5 1.1851526004436805 +5 17581 1 426.0 1.2083890532953205 5 17581 2 233.25 1.1436871765942431 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q39b.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q39b.sql.out new file mode 100644 index 0000000000000..a250c0afd3d44 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q39b.sql.out @@ -0,0 +1,15 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1 13157 1 260.5 1.5242630430075292 1 13157 2 413.5 1.0422561797285326 +1 15441 1 295.0 1.597573089984185 1 15441 2 296.0 1.534664092337063 +1 16895 1 197.25 1.6216484458744376 1 16895 2 201.33333333333334 1.001838963026182 +2 5915 1 231.25 1.8705235468767831 2 5915 2 407.0 1.0885509926353207 +2 11073 1 274.0 1.6770433716259476 2 11073 2 359.0 1.0133649606830526 +3 3315 1 271.75 1.555976998814345 3 3315 2 393.75 1.0196319345405949 +3 3393 1 260.0 1.5009563026568116 3 3393 2 470.25 1.129275872154205 +3 10427 1 241.33333333333334 1.5634035191786233 3 10427 2 381.25 1.0623056061004696 +4 16211 1 257.6666666666667 1.6381074811154002 4 16211 2 352.25 1.055236934125639 +5 8665 1 257.75 1.8119629759287612 5 8665 2 368.0 1.0243808356311228 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q4.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q4.sql.out new file mode 100644 index 0000000000000..a9706ef782fde --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q4.sql.out @@ -0,0 +1,7 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAMHOLAAAA Terri Cook N COSTA RICA NULL Terri.Cook@Vz02fJPUlPO.edu +AAAAAAAANBECBAAA Michael Lombardi Y ZIMBABWE NULL Michael.Lombardi@J.com diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q40.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q40.sql.out new file mode 100644 index 0000000000000..db2d76e5d019e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q40.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +TN AAAAAAAAAAEAAAAA 51.75 -61.82 +TN AAAAAAAAAANDAAAA 131.09 324.87 +TN AAAAAAAAAAPBAAAA 50.90 146.62 +TN AAAAAAAAACACAAAA 113.63 89.13 +TN AAAAAAAAACGCAAAA 53.39 37.02 +TN AAAAAAAAACKCAAAA 65.49 99.60 +TN AAAAAAAAACPCAAAA 0.00 73.18 +TN AAAAAAAAADACAAAA 132.24 0.00 +TN AAAAAAAAADBBAAAA 211.20 0.00 +TN AAAAAAAAADNAAAAA 202.46 0.00 +TN AAAAAAAAAEFCAAAA 45.57 38.18 +TN AAAAAAAAAEGEAAAA -19.57 109.62 +TN AAAAAAAAAELBAAAA 34.98 144.32 +TN AAAAAAAAAFBEAAAA 130.26 3.53 +TN AAAAAAAAAFFDAAAA 27.12 104.61 +TN AAAAAAAAAFJDAAAA -41.08 0.00 +TN AAAAAAAAAFLDAAAA 89.24 0.00 +TN AAAAAAAAAFODAAAA 16.39 0.00 +TN AAAAAAAAAGEBAAAA 146.51 20.38 +TN AAAAAAAAAGIDAAAA 2.84 0.00 +TN AAAAAAAAAGLDAAAA -40.82 75.88 +TN AAAAAAAAAGPCAAAA 136.76 190.61 +TN AAAAAAAAAHBAAAAA 66.92 219.76 +TN AAAAAAAAAHPBAAAA 148.73 -316.99 +TN AAAAAAAAAIADAAAA 230.28 132.32 +TN AAAAAAAAAIGDAAAA 190.52 0.00 +TN AAAAAAAAAINCAAAA 207.61 208.83 +TN AAAAAAAAAJHDAAAA -39.87 215.04 +TN AAAAAAAAAJIDAAAA -351.64 142.38 +TN AAAAAAAAAJOAAAAA 58.56 0.00 +TN AAAAAAAAAKABAAAA 110.04 0.00 +TN AAAAAAAAAKFEAAAA -284.41 91.42 +TN AAAAAAAAALAAAAAA -4.78 133.34 +TN AAAAAAAAALDCAAAA 39.16 0.00 +TN AAAAAAAAALFEAAAA 27.83 212.44 +TN AAAAAAAAALIDAAAA 0.00 118.88 +TN AAAAAAAAALOAAAAA 4.18 0.00 +TN AAAAAAAAALOBAAAA 0.00 0.00 +TN AAAAAAAAALPAAAAA -30.21 10.95 +TN AAAAAAAAAMJBAAAA -24.54 85.92 +TN AAAAAAAAANLCAAAA 84.38 67.38 +TN AAAAAAAAANPAAAAA 74.98 3.48 +TN AAAAAAAAANPDAAAA 484.43 90.58 +TN AAAAAAAAAOFAAAAA 0.00 91.90 +TN AAAAAAAAAOMDAAAA 188.82 29.34 +TN AAAAAAAAAPPBAAAA 0.00 146.30 +TN AAAAAAAABABCAAAA 252.04 27.49 +TN AAAAAAAABADAAAAA 27.80 35.00 +TN AAAAAAAABAGAAAAA 17.30 2.53 +TN AAAAAAAABCCCAAAA -254.31 0.00 +TN AAAAAAAABDCBAAAA 38.34 101.58 +TN AAAAAAAABECDAAAA 223.68 112.52 +TN AAAAAAAABEMCAAAA 0.00 160.01 +TN AAAAAAAABGECAAAA 0.00 224.53 +TN AAAAAAAABGIBAAAA 105.83 0.00 +TN AAAAAAAABGNCAAAA 0.00 72.03 +TN AAAAAAAABHACAAAA 0.00 0.00 +TN AAAAAAAABHPCAAAA 97.28 190.95 +TN AAAAAAAABJFEAAAA 70.68 38.07 +TN AAAAAAAABJMAAAAA 31.23 140.56 +TN AAAAAAAABKEBAAAA 133.61 92.51 +TN AAAAAAAABLBDAAAA 50.89 37.68 +TN AAAAAAAABLEDAAAA -119.59 0.00 +TN AAAAAAAABLOCAAAA 112.88 62.95 +TN AAAAAAAABMCBAAAA 16.30 70.83 +TN AAAAAAAABMOBAAAA 54.02 130.38 +TN AAAAAAAABNEBAAAA 48.98 -803.52 +TN AAAAAAAABOKDAAAA 0.00 135.88 +TN AAAAAAAABONAAAAA 30.07 213.69 +TN AAAAAAAABPPAAAAA 4.77 91.54 +TN AAAAAAAACACEAAAA 244.10 0.00 +TN AAAAAAAACALAAAAA -832.53 0.00 +TN AAAAAAAACALBAAAA 0.00 -641.98 +TN AAAAAAAACBDBAAAA 45.72 145.13 +TN AAAAAAAACBHBAAAA 102.42 0.00 +TN AAAAAAAACBICAAAA 0.95 85.93 +TN AAAAAAAACBJCAAAA 89.33 143.00 +TN AAAAAAAACBNAAAAA 198.92 133.15 +TN AAAAAAAACCABAAAA 61.26 65.34 +TN AAAAAAAACCJBAAAA 4.84 0.00 +TN AAAAAAAACCPDAAAA 280.66 283.09 +TN AAAAAAAACDECAAAA -667.40 -693.66 +TN AAAAAAAACDIBAAAA 3.73 148.26 +TN AAAAAAAACEACAAAA 0.00 -978.31 +TN AAAAAAAACEHDAAAA 66.23 123.74 +TN AAAAAAAACEICAAAA 30.14 117.63 +TN AAAAAAAACEODAAAA 0.00 1.28 +TN AAAAAAAACFKDAAAA 0.00 325.59 +TN AAAAAAAACFLCAAAA 13.34 76.65 +TN AAAAAAAACGCBAAAA -1636.68 167.66 +TN AAAAAAAACGCDAAAA 0.00 8.58 +TN AAAAAAAACGJDAAAA -422.98 490.30 +TN AAAAAAAACGMAAAAA 142.04 196.75 +TN AAAAAAAACHCDAAAA -936.02 96.99 +TN AAAAAAAACIICAAAA -175.56 40.39 +TN AAAAAAAACIJBAAAA 0.00 16.71 +TN AAAAAAAACIMDAAAA 266.31 0.00 +TN AAAAAAAACJBEAAAA -5.33 22.57 +TN AAAAAAAACJDDAAAA 72.37 0.00 +TN AAAAAAAACJGCAAAA 4.59 237.09 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q41.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q41.sql.out new file mode 100644 index 0000000000000..aaaa39f72f03a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q41.sql.out @@ -0,0 +1,10 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +antieingeseableought +eingeseeseese +eseeingeseableought +oughtoughtablecally +priantin stoughtought diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q42.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q42.sql.out new file mode 100644 index 0000000000000..1b17f237b3d06 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q42.sql.out @@ -0,0 +1,16 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +2000 7 Home 494603.64 +2000 2 Men 390852.57 +2000 4 Shoes 378230.23 +2000 3 Children 359411.01 +2000 9 Books 319480.51 +2000 10 Electronics 317086.16 +2000 8 Sports 287853.86 +2000 6 Jewelry 278786.18 +2000 1 Women 245897.86 +2000 5 Music 189405.76 +2000 NULL NULL 39507.19 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q43.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q43.sql.out new file mode 100644 index 0000000000000..38e7f9316f5c0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q43.sql.out @@ -0,0 +1,11 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +able AAAAAAAACAAAAAAA 495537.85 454457.37 480383.13 468495.25 496050.12 471996.47 510311.54 +ation AAAAAAAAHAAAAAAA 485290.49 465106.17 462449.17 478799.15 521647.09 474980.87 484757.85 +bar AAAAAAAAKAAAAAAA 510374.60 458247.16 464054.12 473015.97 487045.12 495531.52 502011.00 +eing AAAAAAAAIAAAAAAA 513799.80 464451.67 440681.32 501857.36 476201.82 452754.56 481703.98 +ese AAAAAAAAEAAAAAAA 529524.71 460191.13 492178.33 458067.77 488508.48 477658.66 490178.23 +ought AAAAAAAABAAAAAAA 464514.26 460448.28 465249.85 482655.67 474754.90 479860.76 474064.44 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q44.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q44.sql.out new file mode 100644 index 0000000000000..959a3fe521152 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q44.sql.out @@ -0,0 +1,15 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1 oughtantiought eingbarantiationought +2 barprin steing callyableationpri +3 ableoughtantipriought ablebarcallypriought +4 oughtcallyoughtation callyn steseable +5 oughteseesecally ableoughtbarn st +6 eingationationanti callyableoughtn st +7 ablecallyought barbarcallyableought +8 n steingprieseought eseeingationeseought +9 priprieinganti bareseprieseought +10 oughtcallybaroughtought eingcallyantipri diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q45.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q45.sql.out new file mode 100644 index 0000000000000..c807bbb5771ed --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q45.sql.out @@ -0,0 +1,25 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +12150 Montezuma 11.23 +20525 Ryan 57.27 +21087 Macedonia 23.20 +21933 Mount Pleasant 51.15 +22924 Belleville 25.42 +24289 NULL 17.89 +26098 Five Points 27.24 +45281 Bethel 113.11 +48014 Clifton 20.69 +48828 Greenwood 58.48 +50411 Cedar Grove 70.45 +51933 Mount Pleasant 23.43 +54536 Friendship 123.37 +58605 Antioch 95.49 +65817 Bridgeport 47.98 +65867 Riceville 11.06 +69843 Oakland 11.82 +71944 Gravel Hill 23.40 +76971 Wilson 111.77 +78048 Salem 188.36 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q46.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q46.sql.out new file mode 100644 index 0000000000000..b9f5ef0471ae9 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q46.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL Antioch Florence 19942 6675.56 -20224.05 +NULL NULL Antioch Jamestown 189909 188.04 4440.84 +NULL NULL Antioch Sulphur Springs 211628 0.00 1690.47 +NULL NULL Arlington Leesburg 112878 1749.37 -4137.65 +NULL NULL Arlington Mount Olive 186130 3625.05 -9174.03 +NULL NULL Arthur Bridgeport 89194 1207.33 -3747.04 +NULL NULL Ashland Antioch 25749 1184.60 -22285.21 +NULL NULL Ashland Cherry Valley 92172 2534.66 -5553.97 +NULL NULL Ashland Superior 91601 7305.88 -10487.66 +NULL NULL Belmont Hamilton 1029 935.93 -1026.72 +NULL NULL Bethel Oakwood 179326 2593.05 -13662.26 +NULL NULL Bethel Woodland 95034 407.35 -15767.89 +NULL NULL Bridgeport Riverside 23264 89.15 -6827.82 +NULL NULL Bridgeport Wildwood 90648 4796.33 -6445.14 +NULL NULL Brownsville Oak Grove 114024 2720.05 -18657.62 +NULL NULL Brunswick Valley View 153419 0.00 -13840.52 +NULL NULL Buena Vista Clearview 109316 257.04 -11539.44 +NULL NULL Buena Vista Deerfield 17940 0.00 -1431.61 +NULL NULL Bunker Hill Colfax 45044 2803.00 2310.74 +NULL NULL Cedar Grove Brownsville 17159 865.78 -9843.40 +NULL NULL Centerville Fairfield 11688 604.66 -7431.13 +NULL NULL Centerville Fisher 24700 330.27 -8639.65 +NULL NULL Centerville Glenwood 27267 0.00 -7024.69 +NULL NULL Clifton Springhill 190087 2987.19 -13925.26 +NULL NULL Clinton Edgewood 84531 8561.97 -28472.50 +NULL NULL Clinton Oakland 179459 879.23 -875.68 +NULL NULL Concord Lakewood 81197 1244.92 866.30 +NULL NULL Crossroads Mount Olive 229868 223.72 139.81 +NULL NULL Ellsworth Shiloh 103474 1412.84 -8861.07 +NULL NULL Enterprise Five Forks 234468 2248.50 -7218.60 +NULL NULL Enterprise Hillcrest 216432 529.37 -2245.35 +NULL NULL Fairfield Arlington 75875 565.09 -14322.52 +NULL NULL Fairview Friendship 172926 1897.88 -11040.49 +NULL NULL Fairview Mount Olive 127246 7793.58 -12492.10 +NULL NULL Farmington Springfield 222902 1213.28 -1240.73 +NULL NULL Fayette Kingston 32825 0.00 -6315.77 +NULL NULL Ferguson Belmont 238393 11864.37 -26448.01 +NULL NULL Five Forks Florence 224673 1507.52 -1549.30 +NULL NULL Five Forks Newport 131275 3957.94 -8960.42 +NULL NULL Five Points Brownsville 122401 166.11 -12890.14 +NULL NULL Five Points Shady Grove 127068 41.49 -3127.13 +NULL NULL Five Points Union 233626 243.82 -13369.20 +NULL NULL Florence Deerfield 181713 2673.95 -11875.68 +NULL NULL Forest Hills Shiloh 76517 2206.26 -20795.18 +NULL NULL Friendship Hamilton 49257 2542.47 14.28 +NULL NULL Friendship Lebanon 230509 7322.26 -3897.95 +NULL NULL Friendship Springdale 55155 187.12 -8555.50 +NULL NULL Glendale Liberty 127860 179.92 -9645.97 +NULL NULL Glendale Springfield 107410 236.04 -11521.38 +NULL NULL Glendale Stringtown 31383 0.00 -5195.83 +NULL NULL Glendale Union 3484 793.55 5358.57 +NULL NULL Glenwood Arlington 227159 236.19 -10529.52 +NULL NULL Glenwood Belmont 72581 1808.15 -7585.34 +NULL NULL Gravel Hill Farmington 62500 0.00 -1479.92 +NULL NULL Greenville Bethel 84058 225.62 -1544.42 +NULL NULL Greenville Marion 221320 669.39 -7120.33 +NULL NULL Greenville Spring Hill 47915 11384.61 -24018.75 +NULL NULL Greenwood Pleasant Hill 4709 822.69 -17295.72 +NULL NULL Greenwood Springdale 204972 263.33 3302.02 +NULL NULL Hamilton Cedar Grove 130494 0.00 -10426.22 +NULL NULL Hamilton Jackson 170132 5140.01 -20602.97 +NULL NULL Hamilton Jamestown 153509 2156.76 -9529.97 +NULL NULL Hardy Harmony 94856 6219.17 -11811.92 +NULL NULL Harmony Plainview 226229 9483.18 -21260.39 +NULL NULL Highland Park Clinton 75671 818.52 -17086.12 +NULL NULL Hillcrest Arlington 131958 608.34 -10931.94 +NULL NULL Hillcrest Berea 60399 764.60 -10177.35 +NULL NULL Hillcrest Harmony 196512 1.70 -9338.87 +NULL NULL Hillcrest Mount Olive 129867 7792.52 -10929.36 +NULL NULL Hillcrest Wilson 33115 4120.87 -17407.74 +NULL NULL Indian Village Golden 98530 284.90 -11785.95 +NULL NULL Jackson Florence 167217 254.54 -18883.41 +NULL NULL Jackson Pleasant Grove 172765 5569.64 -7776.84 +NULL NULL Jamestown Spring Valley 125125 4067.22 -8344.61 +NULL NULL Kingston Farmington 196244 0.00 -4449.03 +NULL NULL Kingston Highland 45785 5257.12 -21282.22 +NULL NULL Lakeside Ashley 233189 1043.83 -3335.36 +NULL NULL Lakeside Highland Park 194552 677.94 -11488.01 +NULL NULL Lakeside Jenkins 71469 434.92 -9825.80 +NULL NULL Lakeside Lakewood 181131 42.20 3148.24 +NULL NULL Lakeside Riverview 183085 116.06 -1892.62 +NULL NULL Lakeside Summit 200887 554.47 -8140.32 +NULL NULL Lakeview Greenfield 230719 0.00 -3529.16 +NULL NULL Lancaster Waterloo 197637 976.52 -11255.13 +NULL NULL Langdon Buena Vista 12247 1626.56 -12689.74 +NULL NULL Lebanon Mount Zion 2701 11616.00 -11748.63 +NULL NULL Lincoln Greenville 126303 0.00 -18688.05 +NULL NULL Louisville Kingston 215207 986.85 -9348.55 +NULL NULL Macedonia Ashland 81673 142.17 -56.02 +NULL NULL Macedonia Guthrie 132070 1459.41 -9746.92 +NULL NULL Maple Grove Five Points 40712 4051.76 -4016.52 +NULL NULL Marion Lakewood 203438 8203.21 -18846.98 +NULL NULL Midway Summerfield 117504 1027.47 -21021.42 +NULL NULL Mount Olive Salem 34593 910.10 -6410.67 +NULL NULL Mount Olive Springdale 73957 2276.08 -18160.48 +NULL NULL Mount Vernon Crossroads 216545 0.00 -10080.14 +NULL NULL Mount Vernon White Oak 25937 2221.06 -12575.90 +NULL NULL Murphy Buena Vista 17149 0.00 -39.90 +NULL NULL New Hope Farmington 38760 6931.68 -10392.42 +NULL NULL New Hope Hillcrest 119378 625.62 -15891.66 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q47.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q47.sql.out new file mode 100644 index 0000000000000..5cd532e0008c8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q47.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Shoes exportiedu pack #x ation Unknown 1999 3 5607.487500 2197.48 3271.66 2831.67 +Shoes exportiedu pack #x ought Unknown 1999 2 5643.938333 2393.31 4463.11 2652.44 +Shoes exportiedu pack #x able Unknown 1999 4 5640.362500 2416.57 3348.90 2987.78 +Men amalgimporto #x ought Unknown 1999 6 4702.116667 1534.95 2666.37 2514.13 +Men edu packimporto #x eing Unknown 1999 7 5330.618333 2218.63 3182.74 7436.83 +Shoes exportiedu pack #x ese Unknown 1999 4 5338.852500 2233.60 3470.43 2832.41 +Music exportischolar #x ese Unknown 1999 5 5139.465000 2034.96 3149.72 3648.17 +Men exportiimporto #x eing Unknown 1999 7 5748.707500 2645.88 3432.67 7646.91 +Men importoimporto #x bar Unknown 1999 3 4915.190000 1815.92 2884.81 2956.00 +Women amalgamalg #x ation Unknown 1999 6 4586.300000 1508.07 2992.12 3059.37 +Shoes exportiedu pack #x eing Unknown 1999 4 5374.032500 2322.80 2484.06 3313.69 +Men importoimporto #x ese Unknown 1999 6 4596.057500 1577.69 2457.43 2439.68 +Men edu packimporto #x ation Unknown 1999 3 5839.670833 2825.07 3157.46 3531.36 +Men edu packimporto #x able Unknown 1999 4 5342.149167 2347.82 2787.31 3588.16 +Shoes exportiedu pack #x ought Unknown 1999 3 5643.938333 2652.44 2393.31 3008.88 +Men exportiimporto #x ought Unknown 1999 3 5475.719167 2515.22 2709.11 2702.85 +Shoes exportiedu pack #x ese Unknown 1999 6 5338.852500 2388.17 2832.41 4216.10 +Shoes exportiedu pack #x bar Unknown 1999 2 5065.706667 2119.85 3439.05 2640.59 +Shoes amalgedu pack #x ation Unknown 1999 2 4713.009167 1774.19 3366.21 1940.05 +Shoes exportiedu pack #x eing Unknown 1999 3 5374.032500 2484.06 2994.92 2322.80 +Men edu packimporto #x able Unknown 1999 7 5342.149167 2454.72 2766.42 7665.06 +Men amalgimporto #x ese Unknown 1999 4 4741.993333 1879.47 3419.59 2634.29 +Music exportischolar #x able Unknown 1999 4 4723.742500 1866.30 2384.82 2931.55 +Shoes importoedu pack #x ation Unknown 1999 4 4732.205000 1875.08 2686.46 3422.03 +Children importoexporti #x ese Unknown 1999 6 4849.967500 2002.59 2590.23 3380.67 +Men importoimporto #x ation Unknown 1999 6 4920.419167 2077.36 3402.55 3347.44 +Men importoimporto #x ese Unknown 1999 4 4596.057500 1762.17 2728.14 2457.43 +Children edu packexporti #x eing Unknown 1999 4 4739.001667 1923.04 2309.91 2849.64 +Music exportischolar #x ought Unknown 1999 3 4816.848333 2010.47 2539.57 2940.38 +Shoes edu packedu pack #x ese Unknown 1999 2 4707.697500 1903.70 2693.49 3474.34 +Shoes importoedu pack #x ought Unknown 1999 2 4443.995833 1642.32 3972.50 2319.04 +Men edu packimporto #x ation Unknown 1999 6 5839.670833 3053.78 3151.27 3622.65 +Shoes exportiedu pack #x ation Unknown 1999 4 5607.487500 2831.67 2197.48 4187.54 +Men edu packimporto #x ought Unknown 1999 3 5598.894167 2824.13 3154.80 3135.60 +Shoes amalgedu pack #x ation Unknown 1999 3 4713.009167 1940.05 1774.19 2496.18 +Men exportiimporto #x ought Unknown 1999 4 5475.719167 2702.85 2515.22 4364.56 +Shoes amalgedu pack #x ese Unknown 1999 4 4596.537500 1825.00 2777.56 3234.34 +Music importoscholar #x able Unknown 1999 6 4332.550833 1563.26 2484.37 2460.11 +Men edu packimporto #x ese Unknown 1999 2 5102.436667 2333.32 3417.85 2536.68 +Men exportiimporto #x ought Unknown 1999 2 5475.719167 2709.11 4740.72 2515.22 +Men exportiimporto #x ought Unknown 1999 6 5475.719167 2723.08 4364.56 4000.42 +Shoes exportiedu pack #x ese Unknown 1999 2 5338.852500 2587.72 3837.18 3470.43 +Children exportiexporti #x eing Unknown 1999 6 4426.704167 1683.86 2186.50 1970.76 +Shoes exportiedu pack #x ought Unknown 1999 6 5643.938333 2902.72 3046.01 3994.18 +Shoes exportiedu pack #x eing Unknown 1999 6 5374.032500 2637.31 3313.69 3158.65 +Men importoimporto #x ation Unknown 1999 3 4920.419167 2183.89 3598.63 2841.78 +Men edu packimporto #x eing Unknown 1999 3 5330.618333 2602.65 2613.45 2624.00 +Women edu packamalg #x eing Unknown 1999 6 4366.627500 1640.10 3159.43 2882.91 +Music exportischolar #x ese Unknown 1999 2 5139.465000 2413.63 4627.89 2773.28 +Men exportiimporto #x ese Unknown 1999 2 4929.611667 2207.89 3940.06 2246.76 +Women edu packamalg #x ese Unknown 1999 2 4551.444167 1833.08 3729.30 2940.96 +Men edu packimporto #x eing Unknown 1999 2 5330.618333 2613.45 3859.71 2602.65 +Children edu packexporti #x ese Unknown 1999 2 4640.935833 1927.95 3524.16 2584.03 +Children exportiexporti #x ese Unknown 1999 7 4565.828333 1853.57 2846.40 6447.36 +Women edu packamalg #x ese Unknown 1999 6 4551.444167 1840.98 2478.75 3174.74 +Men edu packimporto #x eing Unknown 1999 4 5330.618333 2624.00 2602.65 3722.18 +Children importoexporti #x able Unknown 1999 3 5024.325000 2319.29 2691.31 2350.53 +Children edu packexporti #x ought Unknown 1999 2 4836.831667 2132.80 3984.13 2182.12 +Children edu packexporti #x ese Unknown 1999 4 4640.935833 1950.03 2584.03 2919.51 +Shoes importoedu pack #x bar Unknown 1999 3 4791.740833 2100.87 2211.40 3278.75 +Men edu packimporto #x ation Unknown 1999 5 5839.670833 3151.27 3531.36 3053.78 +Men exportiimporto #x ese Unknown 1999 3 4929.611667 2246.76 2207.89 3208.14 +Men edu packimporto #x ation Unknown 1999 2 5839.670833 3157.46 3425.68 2825.07 +Men exportiimporto #x bar Unknown 1999 5 5311.965833 2631.11 2819.12 3022.54 +Music importoscholar #x ation Unknown 1999 2 4388.770833 1714.25 2228.69 2407.93 +Shoes importoedu pack #x ese Unknown 1999 2 4989.784167 2315.46 4430.70 2976.26 +Children importoexporti #x able Unknown 1999 4 5024.325000 2350.53 2319.29 2984.92 +Women amalgamalg #x ation Unknown 1999 3 4586.300000 1917.37 3282.12 2557.46 +Children edu packexporti #x ought Unknown 1999 3 4836.831667 2182.12 2132.80 3466.25 +Shoes exportiedu pack #x able Unknown 1999 5 5640.362500 2987.78 2416.57 3437.87 +Men importoimporto #x ought Unknown 1999 4 5071.261667 2425.64 2696.60 2641.17 +Shoes importoedu pack #x ought Unknown 1999 6 4443.995833 1801.87 2297.76 3078.40 +Children importoexporti #x ought Unknown 1999 2 4480.353333 1838.88 3250.53 2375.04 +Men edu packimporto #x ese Unknown 1999 7 5102.436667 2464.06 2798.80 6978.09 +Shoes exportiedu pack #x ought Unknown 1999 4 5643.938333 3008.88 2652.44 3046.01 +Men importoimporto #x ought Unknown 1999 2 5071.261667 2438.50 4282.24 2696.60 +Children edu packexporti #x ation Unknown 1999 6 4525.318333 1896.34 2883.95 2727.36 +Children amalgexporti #x ese Unknown 1999 4 4212.889167 1588.72 2296.49 3077.12 +Shoes importoedu pack #x eing Unknown 1999 4 4884.260000 2261.01 2502.36 3210.47 +Men exportiimporto #x eing Unknown 1999 3 5748.707500 3132.90 3747.34 3768.23 +Shoes exportiedu pack #x able Unknown 1999 2 5640.362500 3027.85 4285.15 3348.90 +Music edu packscholar #x ought Unknown 1999 3 4490.221667 1883.06 2321.31 2211.82 +Children exportiexporti #x able Unknown 1999 2 4423.506667 1816.43 3192.19 3334.11 +Music amalgscholar #x eing Unknown 1999 6 4894.035000 2288.21 3156.77 2845.45 +Music amalgscholar #x able Unknown 1999 5 4829.768333 2225.46 3380.94 2782.42 +Women edu packamalg #x ation Unknown 1999 3 4268.562500 1665.68 1758.65 2082.99 +Children exportiexporti #x bar Unknown 1999 3 4267.374167 1667.03 2579.77 2802.05 +Men exportiimporto #x able Unknown 1999 3 4987.821667 2387.78 2962.40 2928.83 +Women importoamalg #x bar Unknown 1999 3 4309.575833 1710.42 2004.22 2742.41 +Shoes exportiedu pack #x ought Unknown 1999 5 5643.938333 3046.01 3008.88 2902.72 +Music exportischolar #x bar Unknown 1999 4 4518.543333 1922.23 2683.26 2227.02 +Children amalgexporti #x bar Unknown 1999 3 4441.555000 1846.53 3827.47 3623.30 +Men exportiimporto #x eing Unknown 1999 5 5748.707500 3156.57 3768.23 3432.67 +Music importoscholar #x ese Unknown 1999 2 4302.831667 1712.70 3561.46 2414.41 +Women amalgamalg #x eing Unknown 1999 7 4494.382500 1904.98 2308.58 5603.14 +Women amalgamalg #x able Unknown 1999 4 4582.713333 1994.71 2408.40 2321.48 +Music edu packscholar #x able Unknown 1999 3 4426.623333 1840.51 2707.80 3147.10 +Shoes importoedu pack #x bar Unknown 1999 2 4791.740833 2211.40 3912.90 2100.87 +Shoes exportiedu pack #x able Unknown 1999 7 5640.362500 3062.31 3437.87 6376.30 +Children amalgexporti #x eing Unknown 1999 3 4733.152500 2155.79 2710.50 2685.74 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q48.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q48.sql.out new file mode 100644 index 0000000000000..c7fa3497c93f8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q48.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +28312 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q49.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q49.sql.out new file mode 100644 index 0000000000000..a1933e05565ca --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q49.sql.out @@ -0,0 +1,37 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +catalog 16735 0.50505050505050505051 1 1 +catalog 12633 0.69662921348314606742 2 2 +catalog 13967 0.70000000000000000000 3 3 +catalog 12819 0.70129870129870129870 4 8 +catalog 16155 0.72043010752688172043 5 4 +catalog 17681 0.75268817204301075269 6 5 +catalog 5975 0.76404494382022471910 7 6 +catalog 11451 0.76744186046511627907 8 7 +catalog 1689 0.80219780219780219780 9 9 +catalog 10311 0.81818181818181818182 10 10 +store 5111 0.78947368421052631579 1 1 +store 11073 0.83505154639175257732 2 3 +store 14429 0.84782608695652173913 3 2 +store 15927 0.86419753086419753086 4 4 +store 10171 0.86868686868686868687 5 5 +store 12783 0.88775510204081632653 6 6 +store 11075 0.89743589743589743590 7 7 +store 12889 0.95652173913043478261 8 8 +store 1939 0.99000000000000000000 9 9 +store 12975 1.00000000000000000000 10 10 +store 10455 1.00000000000000000000 10 10 +store 4333 1.00000000000000000000 10 10 +web 10485 0.48863636363636363636 1 1 +web 4483 0.52688172043010752688 2 2 +web 8833 0.58241758241758241758 3 3 +web 1165 0.61458333333333333333 4 4 +web 17197 0.73076923076923076923 5 5 +web 10319 0.73469387755102040816 6 6 +web 13159 0.75257731958762886598 7 7 +web 9629 0.77894736842105263158 8 8 +web 5909 0.78378378378378378378 9 9 +web 7057 0.86746987951807228916 10 10 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q5.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q5.sql.out new file mode 100644 index 0000000000000..0f752c2136fa6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q5.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 113832751.85 3036754.81 -31846972.27 +catalog channel NULL 39549896.80 890805.91 -4700833.96 +catalog channel catalog_pageAAAAAAAAAAABAAAA 217002.51 0.00 13655.90 +catalog channel catalog_pageAAAAAAAAABABAAAA 79307.07 0.00 -3323.53 +catalog channel catalog_pageAAAAAAAAACABAAAA 109295.76 3505.20 8275.32 +catalog channel catalog_pageAAAAAAAAACJAAAAA 0.00 14.82 -113.03 +catalog channel catalog_pageAAAAAAAAADABAAAA 112264.38 0.00 -11845.75 +catalog channel catalog_pageAAAAAAAAADCBAAAA 111462.44 0.00 2684.17 +catalog channel catalog_pageAAAAAAAAAEABAAAA 60875.92 753.55 991.75 +catalog channel catalog_pageAAAAAAAAAECBAAAA 89060.07 0.00 7186.17 +catalog channel catalog_pageAAAAAAAAAEPAAAAA 0.00 2466.17 -1023.59 +catalog channel catalog_pageAAAAAAAAAFABAAAA 42313.54 0.00 -25310.58 +catalog channel catalog_pageAAAAAAAAAFCBAAAA 74908.61 0.00 -10384.86 +catalog channel catalog_pageAAAAAAAAAFPAAAAA 0.00 523.37 -422.16 +catalog channel catalog_pageAAAAAAAAAGABAAAA 120028.56 0.00 4694.80 +catalog channel catalog_pageAAAAAAAAAGBBAAAA 0.00 541.44 -170.28 +catalog channel catalog_pageAAAAAAAAAGCBAAAA 97228.89 0.00 -7579.73 +catalog channel catalog_pageAAAAAAAAAGPAAAAA 0.00 1814.55 -667.34 +catalog channel catalog_pageAAAAAAAAAHABAAAA 69941.58 0.00 -4444.41 +catalog channel catalog_pageAAAAAAAAAHBBAAAA 0.00 138.66 -128.30 +catalog channel catalog_pageAAAAAAAAAHCBAAAA 73967.17 0.00 -5904.39 +catalog channel catalog_pageAAAAAAAAAHPAAAAA 0.00 10125.01 -3291.39 +catalog channel catalog_pageAAAAAAAAAICBAAAA 95341.98 0.00 -10935.38 +catalog channel catalog_pageAAAAAAAAAIMAAAAA 0.00 247.30 -261.67 +catalog channel catalog_pageAAAAAAAAAJCBAAAA 91289.86 0.00 -38131.57 +catalog channel catalog_pageAAAAAAAAAJPAAAAA 0.00 19.98 -102.55 +catalog channel catalog_pageAAAAAAAAAKABAAAA 0.00 3460.18 -1790.30 +catalog channel catalog_pageAAAAAAAAAKCBAAAA 1409.22 0.00 -2367.32 +catalog channel catalog_pageAAAAAAAAAKPAAAAA 176800.33 2203.74 17358.83 +catalog channel catalog_pageAAAAAAAAALBBAAAA 0.00 1719.45 -223.86 +catalog channel catalog_pageAAAAAAAAALCBAAAA 9622.11 0.00 -995.37 +catalog channel catalog_pageAAAAAAAAALPAAAAA 226923.74 0.00 -17678.34 +catalog channel catalog_pageAAAAAAAAAMCBAAAA 1044.59 0.00 -371.42 +catalog channel catalog_pageAAAAAAAAAMPAAAAA 210026.33 0.00 -4598.43 +catalog channel catalog_pageAAAAAAAAANABAAAA 0.00 4859.01 -620.39 +catalog channel catalog_pageAAAAAAAAANPAAAAA 204079.30 6617.69 -43707.08 +catalog channel catalog_pageAAAAAAAAAOCBAAAA 3093.75 0.00 -3029.40 +catalog channel catalog_pageAAAAAAAAAOPAAAAA 205664.00 5.35 -7249.23 +catalog channel catalog_pageAAAAAAAAAPCBAAAA 11986.70 0.00 5352.16 +catalog channel catalog_pageAAAAAAAAAPPAAAAA 259979.15 0.00 15500.93 +catalog channel catalog_pageAAAAAAAABAABAAAA 205054.92 0.00 -13900.72 +catalog channel catalog_pageAAAAAAAABBABAAAA 123870.49 817.57 -35169.40 +catalog channel catalog_pageAAAAAAAABCABAAAA 110311.76 6.60 -2807.92 +catalog channel catalog_pageAAAAAAAABDABAAAA 54375.62 0.00 -16298.08 +catalog channel catalog_pageAAAAAAAABDCBAAAA 77225.26 0.00 -9404.24 +catalog channel catalog_pageAAAAAAAABDIAAAAA 0.00 2247.56 -536.43 +catalog channel catalog_pageAAAAAAAABDPAAAAA 0.00 13366.54 -4827.89 +catalog channel catalog_pageAAAAAAAABEABAAAA 83653.23 301.44 -14544.63 +catalog channel catalog_pageAAAAAAAABECBAAAA 102256.79 0.00 2795.53 +catalog channel catalog_pageAAAAAAAABEPAAAAA 0.00 4374.58 -3395.35 +catalog channel catalog_pageAAAAAAAABFABAAAA 77015.36 0.00 -4370.15 +catalog channel catalog_pageAAAAAAAABFBBAAAA 0.00 2435.93 -3450.68 +catalog channel catalog_pageAAAAAAAABFCBAAAA 105831.18 0.00 12908.55 +catalog channel catalog_pageAAAAAAAABFPAAAAA 0.00 203.00 -162.37 +catalog channel catalog_pageAAAAAAAABGABAAAA 72193.84 0.00 -10412.85 +catalog channel catalog_pageAAAAAAAABGCBAAAA 98074.44 0.00 -1396.21 +catalog channel catalog_pageAAAAAAAABGIAAAAA 0.00 500.56 -315.89 +catalog channel catalog_pageAAAAAAAABGPAAAAA 0.00 257.80 -125.00 +catalog channel catalog_pageAAAAAAAABHABAAAA 109670.37 0.00 17680.12 +catalog channel catalog_pageAAAAAAAABHBBAAAA 0.00 332.44 -218.41 +catalog channel catalog_pageAAAAAAAABHCBAAAA 73731.28 0.00 -14352.31 +catalog channel catalog_pageAAAAAAAABHPAAAAA 0.00 1366.24 -3867.70 +catalog channel catalog_pageAAAAAAAABICBAAAA 71039.47 0.00 2742.64 +catalog channel catalog_pageAAAAAAAABIPAAAAA 0.00 14223.07 -7591.98 +catalog channel catalog_pageAAAAAAAABJBBAAAA 0.00 96.66 -140.68 +catalog channel catalog_pageAAAAAAAABJCBAAAA 5619.24 0.00 -2531.23 +catalog channel catalog_pageAAAAAAAABJPAAAAA 0.00 3881.66 -2212.48 +catalog channel catalog_pageAAAAAAAABKCBAAAA 5420.99 0.00 -5691.26 +catalog channel catalog_pageAAAAAAAABKPAAAAA 161336.55 0.00 -513.02 +catalog channel catalog_pageAAAAAAAABLBBAAAA 0.00 303.27 -173.59 +catalog channel catalog_pageAAAAAAAABLCBAAAA 201.70 0.00 -393.37 +catalog channel catalog_pageAAAAAAAABLPAAAAA 162028.76 54.34 -54551.60 +catalog channel catalog_pageAAAAAAAABMABAAAA 0.00 163.32 -383.25 +catalog channel catalog_pageAAAAAAAABMBBAAAA 0.00 483.00 -4825.77 +catalog channel catalog_pageAAAAAAAABMCBAAAA 2213.59 0.00 -212.78 +catalog channel catalog_pageAAAAAAAABMPAAAAA 140634.77 138.84 -39261.21 +catalog channel catalog_pageAAAAAAAABNABAAAA 0.00 NULL NULL +catalog channel catalog_pageAAAAAAAABNBBAAAA 0.00 2152.88 -910.95 +catalog channel catalog_pageAAAAAAAABNCBAAAA 21066.23 0.00 5619.74 +catalog channel catalog_pageAAAAAAAABNPAAAAA 168937.50 0.00 -7428.35 +catalog channel catalog_pageAAAAAAAABOABAAAA 0.00 756.28 -521.22 +catalog channel catalog_pageAAAAAAAABOBBAAAA 0.00 289.38 -262.27 +catalog channel catalog_pageAAAAAAAABOCBAAAA 3380.37 0.00 337.45 +catalog channel catalog_pageAAAAAAAABOIAAAAA 0.00 553.08 -124.33 +catalog channel catalog_pageAAAAAAAABOPAAAAA 178060.89 453.33 -39647.59 +catalog channel catalog_pageAAAAAAAABPCBAAAA 308.43 0.00 -6294.87 +catalog channel catalog_pageAAAAAAAABPPAAAAA 230886.45 423.60 -7979.91 +catalog channel catalog_pageAAAAAAAACAABAAAA 143217.90 4739.60 -14933.90 +catalog channel catalog_pageAAAAAAAACABBAAAA 0.00 2902.35 -421.56 +catalog channel catalog_pageAAAAAAAACACBAAAA 0.00 275.93 -770.61 +catalog channel catalog_pageAAAAAAAACBABAAAA 82856.21 0.00 -33285.95 +catalog channel catalog_pageAAAAAAAACCABAAAA 87477.08 0.00 818.09 +catalog channel catalog_pageAAAAAAAACDABAAAA 83602.86 0.00 -33313.03 +catalog channel catalog_pageAAAAAAAACDCBAAAA 58195.74 0.00 -16518.32 +catalog channel catalog_pageAAAAAAAACDPAAAAA 0.00 317.65 -658.75 +catalog channel catalog_pageAAAAAAAACEABAAAA 51882.61 0.00 -18201.87 +catalog channel catalog_pageAAAAAAAACECBAAAA 87686.10 0.00 4983.58 +catalog channel catalog_pageAAAAAAAACEPAAAAA 0.00 9419.81 -3490.18 +catalog channel catalog_pageAAAAAAAACFABAAAA 100092.31 0.00 2640.22 +catalog channel catalog_pageAAAAAAAACFCBAAAA 82085.45 0.00 -6014.66 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q50.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q50.sql.out new file mode 100644 index 0000000000000..b1f112afefd8a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q50.sql.out @@ -0,0 +1,11 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct120 days :bigint> +-- !query output +able 1 666 Cedar Spruce Avenue Suite 10 Midway Williamson County TN 31904 78 53 68 63 103 +ation 1 405 3rd Wy Suite 220 Fairview Williamson County TN 35709 71 65 63 43 97 +bar 1 71 Cedar Blvd Suite B Midway Williamson County TN 31904 79 54 43 67 115 +eing 1 914 Lake 11th Road Suite T Midway Williamson County TN 31904 82 45 56 49 105 +ese 1 120 6th Lane Suite B Midway Williamson County TN 31904 63 52 68 43 108 +ought 1 32 3rd Street Suite 220 Midway Williamson County TN 31904 75 63 51 51 99 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q51.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q51.sql.out new file mode 100644 index 0000000000000..a342736812723 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q51.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +8 2000-01-16 NULL 20.83 35.36 20.83 +8 2000-01-18 83.21 NULL 83.21 20.83 +8 2000-02-11 NULL 69.88 83.21 69.88 +14 2000-01-14 NULL 31.65 183.58 31.65 +14 2000-01-29 NULL 63.92 183.58 63.92 +14 2000-02-10 NULL 103.12 183.58 103.12 +14 2000-02-15 NULL 157.03 183.58 157.03 +14 2000-02-16 NULL 183.40 183.58 183.40 +17 2000-05-24 439.75 NULL 439.75 398.92 +17 2000-06-09 NULL 407.77 439.75 407.77 +17 2000-06-13 NULL 408.53 439.75 408.53 +26 2000-01-14 124.68 NULL 124.68 61.34 +26 2000-01-29 NULL 72.77 124.68 72.77 +26 2000-02-06 126.41 NULL 126.41 72.77 +29 2000-02-05 160.79 NULL 160.79 141.46 +29 2000-02-06 167.35 NULL 167.35 141.46 +29 2000-02-07 214.40 NULL 214.40 141.46 +29 2000-02-15 NULL 164.35 214.40 164.35 +29 2000-02-18 NULL 172.82 214.40 172.82 +32 2000-01-21 59.48 NULL 59.48 6.20 +32 2000-01-25 NULL 10.50 59.48 10.50 +32 2000-02-03 NULL 38.01 59.48 38.01 +32 2000-02-09 109.67 NULL 109.67 38.01 +32 2000-02-23 NULL 43.04 109.67 43.04 +32 2000-02-26 NULL 81.08 109.67 81.08 +32 2000-03-05 NULL 99.43 109.67 99.43 +32 2000-03-06 NULL 99.43 109.67 99.43 +43 2000-04-09 172.09 NULL 172.09 169.55 +43 2000-04-23 257.19 NULL 257.19 207.76 +43 2000-04-24 272.83 NULL 272.83 207.76 +43 2000-05-15 NULL 221.70 272.83 221.70 +44 2000-01-02 75.87 26.58 75.87 26.58 +44 2000-01-05 125.14 NULL 125.14 26.58 +47 2000-02-14 148.00 NULL 148.00 74.22 +61 2000-01-19 140.35 75.64 140.35 75.64 +61 2000-01-28 231.75 NULL 231.75 75.64 +61 2000-02-04 NULL 156.79 231.75 156.79 +61 2000-02-27 NULL 224.70 231.75 224.70 +67 2000-01-01 88.63 46.38 88.63 46.38 +67 2000-01-02 NULL 53.21 88.63 53.21 +67 2000-01-04 185.14 100.85 185.14 100.85 +67 2000-01-29 309.98 NULL 309.98 100.85 +67 2000-02-14 NULL 148.29 309.98 148.29 +67 2000-02-20 NULL 174.01 309.98 174.01 +67 2000-03-05 NULL 199.85 309.98 199.85 +67 2000-03-07 NULL 284.98 309.98 284.98 +74 2000-01-06 71.82 NULL 71.82 9.01 +80 2000-03-16 NULL 67.38 169.88 67.38 +80 2000-04-10 NULL 137.63 169.88 137.63 +80 2000-07-01 309.57 NULL 309.57 273.52 +80 2000-07-02 455.94 NULL 455.94 273.52 +80 2000-07-25 NULL 443.48 455.94 443.48 +80 2000-08-04 NULL 443.48 455.94 443.48 +82 2000-01-01 85.75 64.27 85.75 64.27 +83 2000-01-08 134.96 NULL 134.96 8.10 +83 2000-02-01 NULL 52.54 134.96 52.54 +83 2000-02-27 NULL 76.22 134.96 76.22 +83 2000-03-01 NULL 112.53 134.96 112.53 +83 2000-03-02 248.36 NULL 248.36 112.53 +83 2000-03-05 NULL 118.07 248.36 118.07 +83 2000-03-16 250.29 NULL 250.29 118.07 +83 2000-03-19 NULL 163.65 250.29 163.65 +83 2000-04-07 289.65 NULL 289.65 163.65 +83 2000-04-08 NULL 201.68 289.65 201.68 +83 2000-04-10 357.89 NULL 357.89 201.68 +83 2000-04-20 364.69 NULL 364.69 201.68 +83 2000-04-26 371.15 NULL 371.15 201.68 +83 2000-05-06 NULL 206.29 371.15 206.29 +83 2000-05-27 NULL 228.07 371.15 228.07 +83 2000-06-06 393.25 NULL 393.25 228.07 +83 2000-06-09 NULL 235.88 393.25 235.88 +83 2000-06-10 NULL 299.68 393.25 299.68 +85 2000-01-16 48.36 NULL 48.36 42.45 +85 2000-01-18 NULL 47.47 48.36 47.47 +85 2000-02-02 126.73 NULL 126.73 47.47 +85 2000-04-01 NULL 47.47 126.73 47.47 +85 2000-04-08 NULL 58.91 126.73 58.91 +85 2000-04-15 NULL 69.74 126.73 69.74 +85 2000-05-01 NULL 93.80 126.73 93.80 +86 2000-01-16 NULL 19.92 38.25 19.92 +86 2000-01-17 NULL 34.32 38.25 34.32 +86 2000-05-06 257.78 NULL 257.78 204.60 +86 2000-05-08 NULL 222.06 257.78 222.06 +86 2000-05-12 NULL 235.00 257.78 235.00 +89 2000-02-01 143.30 NULL 143.30 125.19 +89 2000-02-11 273.46 NULL 273.46 125.19 +89 2000-03-01 NULL 131.53 273.46 131.53 +89 2000-03-24 NULL 200.30 273.46 200.30 +89 2000-04-04 NULL 212.25 273.46 212.25 +89 2000-04-05 NULL 265.57 273.46 265.57 +97 2000-01-22 114.93 NULL 114.93 55.81 +97 2000-01-31 NULL 60.35 114.93 60.35 +97 2000-03-07 263.86 NULL 263.86 220.68 +98 2000-02-10 NULL 1.04 42.10 1.04 +98 2000-02-23 44.27 NULL 44.27 1.04 +98 2000-02-24 274.58 NULL 274.58 1.04 +98 2000-02-29 301.60 NULL 301.60 1.04 +98 2000-03-13 309.78 NULL 309.78 1.04 +98 2000-03-17 335.79 NULL 335.79 1.04 +98 2000-03-27 NULL 37.67 335.79 37.67 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q52.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q52.sql.out new file mode 100644 index 0000000000000..568469bf9afa6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q52.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +2000 1002002 importoamalg #x 98247.74 +2000 4001001 amalgedu pack #x 97944.74 +2000 2004002 edu packimporto #x 90901.39 +2000 2004001 edu packimporto #x 89064.36 +2000 9003003 exportimaxi #x 67483.21 +2000 4004002 edu packedu pack #x 64713.03 +2000 2002002 importoimporto #x 61594.80 +2000 2001001 amalgimporto #x 57219.92 +2000 10016008 corpamalgamalg #x 56274.63 +2000 3002001 importoexporti #x 52756.72 +2000 3004001 edu packexporti #x 49336.05 +2000 4003001 exportiedu pack #x 47889.60 +2000 1004001 edu packbrand #x 46738.09 +2000 3003002 exportiexporti #x 46024.13 +2000 3004002 edu packexporti #x 42712.09 +2000 6003005 edu packmaxi #x 42381.36 +2000 9015011 exportibrand #x 41249.43 +2000 5001002 amalgscholar #x 40525.36 +2000 9003006 exportimaxi #x 40252.87 +2000 NULL NULL 39507.19 +2000 5002001 importoscholar #x 38829.75 +2000 3003001 exportiexporti #x 36506.92 +2000 4002001 importoedu pack #x 35994.46 +2000 9013003 exportiunivamalg #x 35549.32 +2000 6003004 exporticorp #x 34314.00 +2000 8010002 univmaxi #x 32664.31 +2000 6005004 scholarcorp #x 32175.90 +2000 7011006 amalgnameless #x 30979.02 +2000 10008016 namelessunivamalg #x 30424.25 +2000 10014001 edu packamalgamalg #x 30406.55 +2000 5003001 exportischolar #x 30278.44 +2000 10003008 exportiunivamalg #x 30127.51 +2000 7002002 importobrand #x 29602.59 +2000 7005006 scholarbrand #x 29319.83 +2000 9012009 importounivamalg #x 29204.86 +2000 1001001 importoexporti #x 28800.60 +2000 7011010 amalgnameless #x 28614.32 +2000 10007004 exportiamalg #x 28600.67 +2000 4001002 amalgedu pack #x 27963.59 +2000 4002001 edu packscholar #x 27934.37 +2000 10016002 corpamalgamalg #x 27695.20 +2000 7015008 scholarnameless #x 27490.44 +2000 8006010 corpnameless #x 27390.35 +2000 6008004 namelesscorp #x 27296.01 +2000 5002002 importoscholar #x 27037.76 +2000 8003006 exportinameless #x 26842.25 +2000 1003001 exportiamalg #x 26808.25 +2000 10001007 amalgunivamalg #x 26031.20 +2000 6011001 amalgbrand #x 25567.26 +2000 1001001 amalgamalg #x 25210.15 +2000 8002008 importonameless #x 24937.40 +2000 5004002 edu packscholar #x 24800.08 +2000 9007003 brandmaxi #x 24694.06 +2000 4002002 importoedu pack #x 23924.37 +2000 9008003 namelessmaxi #x 23871.65 +2000 3001001 amalgexporti #x 23023.19 +2000 10009002 maxiunivamalg #x 22932.13 +2000 8014006 edu packmaxi #x 22164.19 +2000 8008006 namelessnameless #x 22159.19 +2000 1002001 importoamalg #x 21952.87 +2000 10005002 edu packedu pack #x 21623.64 +2000 9016008 corpunivamalg #x 21503.39 +2000 8002007 importonameless #x 21370.99 +2000 7013002 exportinameless #x 21315.92 +2000 5003001 importonameless #x 21140.49 +2000 1003002 exportiamalg #x 20558.37 +2000 7014009 edu packedu pack #x 20518.93 +2000 4003001 corpnameless #x 20345.42 +2000 3002001 edu packedu pack #x 19948.39 +2000 6015004 scholarbrand #x 19806.27 +2000 7015004 scholarnameless #x 19805.86 +2000 10005002 exportiimporto #x 19553.70 +2000 6005003 amalgbrand #x 18383.14 +2000 10013010 exportiamalgamalg #x 18349.28 +2000 10010002 exportiexporti #x 18338.53 +2000 2002001 amalgimporto #x 18298.41 +2000 10009016 maxiunivamalg #x 17942.21 +2000 2002001 exportiexporti #x 17817.83 +2000 4003002 exportiedu pack #x 17709.48 +2000 9004008 edu packmaxi #x 17660.51 +2000 7006005 corpbrand #x 17628.79 +2000 10013006 exportiamalgamalg #x 17579.59 +2000 9002011 brandbrand #x 17177.90 +2000 2001001 exportiexporti #x 16270.97 +2000 6011003 amalgbrand #x 16222.01 +2000 8004002 edu packnameless #x 16054.52 +2000 10008008 importoexporti #x 15563.77 +2000 6002008 importocorp #x 15447.93 +2000 4003001 edu packimporto #x 15313.51 +2000 8015004 scholarmaxi #x 15299.50 +2000 8010001 univmaxi #x 15188.28 +2000 7016008 corpnameless #x 15123.33 +2000 6008001 namelesscorp #x 15096.05 +2000 7009003 maxibrand #x 15073.42 +2000 7015007 scholarnameless #x 14934.32 +2000 1001002 amalgamalg #x 14264.80 +2000 7010005 univnameless #x 14247.64 +2000 9005002 scholarmaxi #x 14062.39 +2000 7012008 importonameless #x 14043.20 +2000 9010002 univunivamalg #x 13922.13 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q53.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q53.sql.out new file mode 100644 index 0000000000000..8528e2da8bfe0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q53.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +132 119.11 258.580000 +132 222.31 258.580000 +132 342.09 258.580000 +132 350.81 258.580000 +612 77.21 301.970000 +612 174.17 301.970000 +612 248.75 301.970000 +612 707.75 301.970000 +650 240.94 329.890000 +650 451.30 329.890000 +483 175.17 341.280000 +483 284.63 341.280000 +483 379.24 341.280000 +483 526.08 341.280000 +39 83.05 347.250000 +39 201.75 347.250000 +39 249.20 347.250000 +39 855.00 347.250000 +659 270.05 357.907500 +659 301.12 357.907500 +659 521.64 357.907500 +487 194.22 363.632500 +487 213.65 363.632500 +487 489.98 363.632500 +487 556.68 363.632500 +423 156.29 372.700000 +423 323.98 372.700000 +423 607.23 372.700000 +872 117.67 375.220000 +872 118.10 375.220000 +872 880.58 375.220000 +556 245.39 386.777500 +556 311.12 386.777500 +556 446.09 386.777500 +556 544.51 386.777500 +179 308.69 388.540000 +179 438.91 388.540000 +274 40.68 390.370000 +274 140.57 390.370000 +274 283.26 390.370000 +274 1096.97 390.370000 +551 154.35 397.355000 +551 327.31 397.355000 +551 696.52 397.355000 +67 271.97 397.382500 +67 305.28 397.382500 +67 621.24 397.382500 +53 110.01 397.517500 +53 250.03 397.517500 +53 468.76 397.517500 +53 761.27 397.517500 +172 154.34 398.030000 +172 347.87 398.030000 +172 439.68 398.030000 +172 650.23 398.030000 +915 210.94 403.120000 +915 359.81 403.120000 +915 678.57 403.120000 +350 30.44 407.737500 +350 328.11 407.737500 +350 339.56 407.737500 +350 932.84 407.737500 +203 164.68 408.595000 +203 305.41 408.595000 +203 735.85 408.595000 +2 239.50 408.602500 +2 289.78 408.602500 +2 696.25 408.602500 +687 70.08 410.057500 +687 80.72 410.057500 +687 368.29 410.057500 +687 1121.14 410.057500 +113 76.68 414.170000 +113 268.47 414.170000 +113 531.42 414.170000 +113 780.11 414.170000 +161 224.11 416.682500 +161 298.25 416.682500 +161 480.46 416.682500 +161 663.91 416.682500 +233 284.33 421.205000 +233 320.60 421.205000 +233 322.01 421.205000 +233 757.88 421.205000 +743 173.26 430.360000 +743 198.59 430.360000 +743 565.04 430.360000 +743 784.55 430.360000 +581 51.95 432.642500 +581 247.90 432.642500 +581 509.07 432.642500 +581 921.65 432.642500 +107 254.35 434.100000 +107 348.41 434.100000 +107 388.70 434.100000 +107 744.94 434.100000 +359 163.98 436.225000 +359 215.17 436.225000 +359 379.98 436.225000 +359 985.77 436.225000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q54.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q54.sql.out new file mode 100644 index 0000000000000..dd5f10b74372c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q54.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q55.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q55.sql.out new file mode 100644 index 0000000000000..9592803da4665 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q55.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +4002001 importoedu pack #x 141955.03 +2001001 amalgimporto #x 115066.00 +2003001 exportiimporto #x 96597.01 +5003002 exportischolar #x 93316.15 +1002001 importoamalg #x 85318.86 +2002001 importoimporto #x 85035.18 +3003001 exportiexporti #x 80017.81 +2004001 edu packimporto #x 68604.08 +9001008 amalgmaxi #x 60900.34 +4004001 edu packedu pack #x 58815.34 +4003002 exportiedu pack #x 56587.37 +6012003 importobrand #x 55151.38 +1001002 amalgamalg #x 54471.07 +3004001 edu packexporti #x 54057.61 +3001001 amalgexporti #x 52690.20 +4003001 exportiedu pack #x 51021.71 +5002002 importoscholar #x 46601.52 +8002007 importonameless #x 45983.92 +6016003 corpbrand #x 44911.30 +1003001 exportiamalg #x 44127.51 +6014003 edu packbrand #x 41310.31 +7014009 edu packnameless #x 40891.96 +7008004 namelessbrand #x 36589.35 +10001016 amalgunivamalg #x 36270.01 +6016004 corpbrand #x 36208.64 +9013011 exportiunivamalg #x 35794.13 +1003002 exportiamalg #x 35113.55 +1004002 edu packamalg #x 34667.61 +7016005 corpnameless #x 33397.41 +7014005 edu packnameless #x 33313.92 +7012010 importonameless #x 32862.88 +7012001 importonameless #x 32815.12 +6001007 amalgcorp #x 31948.28 +3002001 importoexporti #x 31518.29 +8015001 scholarmaxi #x 31235.19 +10012006 importoamalgamalg #x 31123.11 +6004004 edu packcorp #x 29839.27 +10009008 maxiunivamalg #x 29180.04 +5004001 edu packscholar #x 28912.61 +9006011 corpmaxi #x 26493.45 +8014004 edu packmaxi #x 26228.69 +9012008 importounivamalg #x 25354.49 +8015003 scholarmaxi #x 23994.34 +7013009 exportinameless #x 23887.30 +2004002 edu packimporto #x 23778.49 +6005005 scholarcorp #x 23243.88 +5001001 amalgscholar #x 23043.20 +8009007 maxinameless #x 22910.03 +8004009 edu packnameless #x 22679.38 +9016002 corpunivamalg #x 21881.94 +8003001 exportinameless #x 21655.96 +8004004 edu packnameless #x 20920.43 +10010007 univamalgamalg #x 20530.11 +8007004 brandnameless #x 19973.79 +9015005 scholarunivamalg #x 19929.31 +6004002 edu packcorp #x 19890.20 +10004005 edu packunivamalg #x 19819.04 +9014008 edu packunivamalg #x 19700.99 +5003001 exportischolar #x 19595.76 +7007006 brandbrand #x 19541.72 +8009001 maxinameless #x 19474.02 +2002002 importoimporto #x 19249.25 +8003002 exportinameless #x 18931.96 +9016011 corpunivamalg #x 18840.83 +10012011 importoamalgamalg #x 18632.17 +9004005 edu packmaxi #x 17997.32 +7003003 exportibrand #x 16699.26 +9011011 amalgunivamalg #x 16656.62 +10008002 namelessunivamalg #x 16609.49 +10016015 corpamalgamalg #x 16397.21 +8008001 namelessnameless #x 15538.37 +6004007 edu packcorp #x 15474.56 +7006001 corpbrand #x 15365.31 +6010003 univbrand #x 15337.21 +1001001 amalgamalg #x 14813.15 +9012003 importounivamalg #x 14033.76 +8015008 scholarmaxi #x 13408.30 +6002005 importocorp #x 13393.21 +7010009 univnameless #x 13359.47 +10010002 univamalgamalg #x 13288.24 +8014003 edu packmaxi #x 13137.78 +10001009 amalgunivamalg #x 12326.55 +5001002 amalgscholar #x 12319.21 +5002001 importoscholar #x 12023.80 +8005010 scholarnameless #x 12017.53 +6008005 namelesscorp #x 11908.45 +3001002 amalgexporti #x 11679.83 +10008006 namelessunivamalg #x 11503.15 +10014009 edu packamalgamalg #x 11462.45 +10007017 brandunivamalg #x 11188.38 +7002004 importobrand #x 10699.51 +7001006 amalgbrand #x 10166.47 +10004009 edu packunivamalg #x 9913.46 +10002011 importounivamalg #x 9828.64 +3003002 exportiexporti #x 9521.40 +10014017 edu packamalgamalg #x 9150.09 +6002003 importocorp #x 8810.78 +6001005 amalgcorp #x 8582.71 +4001001 amalgedu pack #x 8362.72 +8004002 edu packnameless #x 8010.27 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q56.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q56.sql.out new file mode 100644 index 0000000000000..f6d3a7470781c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q56.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAANMMDAAAA NULL +AAAAAAAAIMPDAAAA 0.00 +AAAAAAAAEGJBAAAA 0.59 +AAAAAAAAOGIDAAAA 3.92 +AAAAAAAAFLGAAAAA 17.01 +AAAAAAAAGPEEAAAA 21.60 +AAAAAAAACJCDAAAA 27.30 +AAAAAAAAAAEDAAAA 38.88 +AAAAAAAACPBEAAAA 46.20 +AAAAAAAAMGKBAAAA 50.79 +AAAAAAAACLHAAAAA 58.52 +AAAAAAAAKJOBAAAA 74.48 +AAAAAAAAGMEDAAAA 78.90 +AAAAAAAADGCCAAAA 84.81 +AAAAAAAALLDAAAAA 95.60 +AAAAAAAAONEDAAAA 100.91 +AAAAAAAAAKMAAAAA 102.75 +AAAAAAAAIJIAAAAA 103.53 +AAAAAAAAMLFDAAAA 112.10 +AAAAAAAAHDDAAAAA 113.62 +AAAAAAAAPBMDAAAA 117.80 +AAAAAAAAPFCDAAAA 119.56 +AAAAAAAALICBAAAA 119.68 +AAAAAAAAKAJAAAAA 126.21 +AAAAAAAAOIAAAAAA 127.82 +AAAAAAAAABDEAAAA 145.52 +AAAAAAAAOCKDAAAA 149.00 +AAAAAAAAEMECAAAA 154.20 +AAAAAAAAGOLDAAAA 172.12 +AAAAAAAAAGLDAAAA 173.46 +AAAAAAAAGBGEAAAA 178.20 +AAAAAAAAKOBDAAAA 183.60 +AAAAAAAAEGGDAAAA 187.32 +AAAAAAAAHDKCAAAA 227.76 +AAAAAAAAFDBEAAAA 236.91 +AAAAAAAAEICCAAAA 241.28 +AAAAAAAAAKAAAAAA 250.16 +AAAAAAAAOMBEAAAA 310.90 +AAAAAAAAANHCAAAA 320.32 +AAAAAAAAIJGAAAAA 336.00 +AAAAAAAAAFAAAAAA 336.28 +AAAAAAAAKFGBAAAA 380.37 +AAAAAAAAEACCAAAA 389.75 +AAAAAAAAIOMAAAAA 418.92 +AAAAAAAAEKODAAAA 448.26 +AAAAAAAAENDAAAAA 456.30 +AAAAAAAAAJGBAAAA 457.68 +AAAAAAAAIBEAAAAA 466.74 +AAAAAAAAOMEEAAAA 469.80 +AAAAAAAAMDIDAAAA 501.28 +AAAAAAAAPANDAAAA 519.27 +AAAAAAAAKKECAAAA 523.27 +AAAAAAAAKPABAAAA 564.00 +AAAAAAAABHMCAAAA 566.58 +AAAAAAAAGMGBAAAA 584.56 +AAAAAAAAGKNCAAAA 614.00 +AAAAAAAAGELBAAAA 625.04 +AAAAAAAAKGGDAAAA 642.63 +AAAAAAAAFLPAAAAA 680.48 +AAAAAAAAGJODAAAA 698.32 +AAAAAAAAIBDBAAAA 702.93 +AAAAAAAAGAFAAAAA 733.20 +AAAAAAAAMPEEAAAA 760.16 +AAAAAAAAMDFCAAAA 776.72 +AAAAAAAANPFEAAAA 778.00 +AAAAAAAAMHEAAAAA 778.27 +AAAAAAAAOHLDAAAA 801.88 +AAAAAAAAGIJDAAAA 815.74 +AAAAAAAAICOCAAAA 817.74 +AAAAAAAAAICAAAAA 823.35 +AAAAAAAAKCNAAAAA 833.31 +AAAAAAAACDGAAAAA 836.90 +AAAAAAAAAAKDAAAA 850.07 +AAAAAAAAFOJAAAAA 859.28 +AAAAAAAAMNLCAAAA 865.29 +AAAAAAAAOINCAAAA 865.60 +AAAAAAAAKGOBAAAA 875.20 +AAAAAAAAOFKAAAAA 882.60 +AAAAAAAAIGMCAAAA 886.89 +AAAAAAAAIIOBAAAA 892.05 +AAAAAAAAGOBBAAAA 909.74 +AAAAAAAAPNNCAAAA 981.72 +AAAAAAAAKPAAAAAA 1006.11 +AAAAAAAANDBCAAAA 1028.34 +AAAAAAAAABCCAAAA 1051.08 +AAAAAAAACJCEAAAA 1055.78 +AAAAAAAACEPCAAAA 1064.00 +AAAAAAAACGFEAAAA 1067.78 +AAAAAAAAMOIAAAAA 1073.80 +AAAAAAAAMIFDAAAA 1104.17 +AAAAAAAAAOHBAAAA 1104.40 +AAAAAAAAMGJBAAAA 1124.12 +AAAAAAAAAEFCAAAA 1134.68 +AAAAAAAAAJIDAAAA 1153.92 +AAAAAAAAFEKDAAAA 1156.54 +AAAAAAAAGIBEAAAA 1165.02 +AAAAAAAAKLBBAAAA 1181.93 +AAAAAAAAOBBAAAAA 1183.08 +AAAAAAAABEEEAAAA 1190.91 +AAAAAAAADHJAAAAA 1202.11 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q57.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q57.sql.out new file mode 100644 index 0000000000000..dc712720d3ac5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q57.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Music amalgscholar #x Mid Atlantic 1999 2 6662.669167 1961.57 4348.07 3386.25 +Shoes amalgedu pack #x North Midwest 1999 2 6493.071667 2044.05 4348.88 3443.20 +Shoes exportiedu pack #x Mid Atlantic 1999 3 7416.141667 2980.15 4654.22 5157.83 +Children importoexporti #x North Midwest 1999 4 6577.143333 2152.15 3291.07 3659.32 +Shoes importoedu pack #x NY Metro 1999 6 6926.960833 2523.33 4014.93 4254.99 +Men importoimporto #x North Midwest 1999 2 6707.315833 2449.22 4311.11 3583.31 +Men amalgimporto #x North Midwest 1999 4 7098.680833 2965.42 3526.45 4923.53 +Men exportiimporto #x NY Metro 1999 2 7146.240000 3013.99 6183.83 5091.17 +Children amalgexporti #x North Midwest 1999 4 6364.467500 2270.79 3330.83 3817.50 +Men edu packimporto #x North Midwest 1999 4 7386.333333 3329.74 3488.01 4860.20 +Men edu packimporto #x North Midwest 1999 2 7386.333333 3347.65 4007.40 3488.01 +Music edu packscholar #x NY Metro 1999 7 6639.040000 2653.55 4219.52 10071.22 +Music amalgscholar #x North Midwest 1999 4 6719.304167 2739.33 3690.54 3872.98 +Men importoimporto #x NY Metro 1999 3 6610.034167 2645.24 3661.14 4282.01 +Music exportischolar #x NY Metro 1999 2 7043.051667 3115.83 4457.95 5258.95 +Men edu packimporto #x North Midwest 1999 3 7386.333333 3488.01 3347.65 3329.74 +Shoes exportiedu pack #x North Midwest 1999 3 7255.790000 3411.07 4194.64 3624.85 +Women exportiamalg #x Mid Atlantic 1999 2 5646.671667 1809.52 4198.70 2172.85 +Music importoscholar #x Mid Atlantic 1999 6 6279.081667 2456.98 4361.44 4256.24 +Children importoexporti #x Mid Atlantic 1999 7 6786.750000 2978.82 3942.59 7809.22 +Music exportischolar #x North Midwest 1999 2 7041.705833 3245.77 3608.31 4127.40 +Shoes importoedu pack #x Mid Atlantic 1999 2 6864.320833 3104.48 3135.52 3606.28 +Shoes importoedu pack #x Mid Atlantic 1999 1 6864.320833 3135.52 14580.30 3104.48 +Children edu packexporti #x Mid Atlantic 1999 5 6511.800833 2785.92 3956.90 3906.63 +Women amalgamalg #x Mid Atlantic 1999 2 6480.683333 2769.69 3353.67 3876.51 +Men exportiimporto #x NY Metro 1999 5 7146.240000 3440.57 3561.11 3971.13 +Women importoamalg #x NY Metro 1999 3 6512.794167 2808.28 3789.51 4335.27 +Men amalgimporto #x NY Metro 1999 2 6720.550000 3018.90 4328.03 3810.74 +Shoes exportiedu pack #x North Midwest 1999 7 7255.790000 3557.87 4937.98 9496.49 +Music exportischolar #x Mid Atlantic 1999 5 6791.260833 3096.02 3918.04 3801.90 +Men exportiimporto #x North Midwest 1999 1 7343.719167 3652.72 13689.13 3984.13 +Men importoimporto #x North Midwest 1999 5 6707.315833 3030.18 4977.33 4620.75 +Children exportiexporti #x NY Metro 1999 2 6386.880833 2717.07 4809.11 3355.48 +Men amalgimporto #x North Midwest 1999 2 7098.680833 3440.27 5293.69 3526.45 +Men importoimporto #x NY Metro 1999 5 6610.034167 2954.71 4282.01 3166.43 +Shoes exportiedu pack #x North Midwest 1999 4 7255.790000 3624.85 3411.07 5169.09 +Men exportiimporto #x North Midwest 1999 5 7343.719167 3727.75 3729.62 4580.93 +Men exportiimporto #x North Midwest 1999 4 7343.719167 3729.62 4033.37 3727.75 +Music edu packscholar #x North Midwest 1999 2 6489.175000 2875.98 4299.82 4028.97 +Men edu packimporto #x NY Metro 1999 1 7202.242500 3614.07 15582.63 4234.79 +Music importoscholar #x North Midwest 1999 2 5816.271667 2229.79 2919.29 4298.41 +Men exportiimporto #x NY Metro 1999 4 7146.240000 3561.11 5091.17 3440.57 +Shoes exportiedu pack #x NY Metro 1999 7 7073.462500 3493.41 4534.31 8701.59 +Music exportischolar #x Mid Atlantic 1999 3 6791.260833 3218.13 3847.87 3918.04 +Shoes edu packedu pack #x NY Metro 1999 4 6203.331667 2631.02 4424.18 4186.85 +Men amalgimporto #x North Midwest 1999 3 7098.680833 3526.45 3440.27 2965.42 +Men edu packimporto #x NY Metro 1999 3 7202.242500 3639.93 4234.79 4016.25 +Children amalgexporti #x North Midwest 1999 2 6364.467500 2825.09 4111.08 3330.83 +Shoes edu packedu pack #x North Midwest 1999 2 6464.239167 2928.99 4233.86 3840.97 +Shoes amalgedu pack #x North Midwest 1999 4 6493.071667 2962.20 3443.20 4212.60 +Music importoscholar #x NY Metro 1999 4 5707.844167 2179.41 3789.16 4317.53 +Shoes exportiedu pack #x Mid Atlantic 1999 1 7416.141667 3892.54 14170.68 4654.22 +Women importoamalg #x NY Metro 1999 5 6512.794167 2991.07 4335.27 4624.86 +Music exportischolar #x NY Metro 1999 4 7043.051667 3521.99 5258.95 4135.21 +Women edu packamalg #x Mid Atlantic 1999 2 6354.045833 2836.23 3719.67 3527.07 +Music amalgscholar #x NY Metro 1999 3 6123.475000 2617.39 3080.43 4919.93 +Shoes amalgedu pack #x Mid Atlantic 1999 7 6674.896667 3178.12 3342.98 8050.81 +Men amalgimporto #x Mid Atlantic 1999 5 6618.534167 3127.28 4291.66 4669.62 +Women amalgamalg #x North Midwest 1999 6 6874.250000 3387.53 4798.69 4329.48 +Women exportiamalg #x Mid Atlantic 1999 3 5646.671667 2172.85 1809.52 4461.31 +Children edu packexporti #x NY Metro 1999 2 6112.954167 2641.54 3567.58 3196.45 +Children amalgexporti #x NY Metro 1999 5 6294.100833 2834.48 3317.60 3803.79 +Women edu packamalg #x NY Metro 1999 5 6027.880000 2575.81 2750.96 4459.01 +Music exportischolar #x North Midwest 1999 6 7041.705833 3589.97 4134.03 4892.26 +Music exportischolar #x North Midwest 1999 4 7041.705833 3593.86 4127.40 4134.03 +Men importoimporto #x NY Metro 1999 6 6610.034167 3166.43 2954.71 3673.39 +Music exportischolar #x North Midwest 1999 1 7041.705833 3608.31 15046.54 3245.77 +Music edu packscholar #x Mid Atlantic 1999 2 6602.385000 3173.44 3434.91 3929.40 +Music amalgscholar #x NY Metro 1999 6 6123.475000 2699.75 4038.47 3330.87 +Children importoexporti #x Mid Atlantic 1999 4 6786.750000 3366.25 3847.60 4259.57 +Men edu packimporto #x Mid Atlantic 1999 1 7230.493333 3811.64 14668.93 4497.31 +Shoes importoedu pack #x Mid Atlantic 1999 5 6864.320833 3449.62 3869.15 3531.93 +Children edu packexporti #x North Midwest 1999 2 6739.498333 3328.01 4986.50 3623.32 +Children importoexporti #x Mid Atlantic 1999 1 6786.750000 3376.55 12504.93 5018.22 +Children edu packexporti #x NY Metro 1999 7 6112.954167 2711.04 3254.04 9465.10 +Shoes importoedu pack #x North Midwest 1999 3 6588.741667 3187.25 4283.21 3212.76 +Men importoimporto #x Mid Atlantic 1999 3 6702.415000 3310.55 3981.06 4901.56 +Men edu packimporto #x North Midwest 1999 1 7386.333333 4007.40 14005.45 3347.65 +Shoes importoedu pack #x North Midwest 1999 4 6588.741667 3212.76 3187.25 3974.78 +Shoes edu packedu pack #x NY Metro 1999 6 6203.331667 2835.78 4186.85 3192.53 +Men exportiimporto #x North Midwest 1999 2 7343.719167 3984.13 3652.72 4033.37 +Men amalgimporto #x NY Metro 1999 4 6720.550000 3364.32 3810.74 4333.58 +Children edu packexporti #x North Midwest 1999 4 6739.498333 3389.03 3623.32 3605.25 +Shoes importoedu pack #x Mid Atlantic 1999 6 6864.320833 3531.93 3449.62 4414.17 +Shoes amalgedu pack #x Mid Atlantic 1999 6 6674.896667 3342.98 4458.26 3178.12 +Children edu packexporti #x Mid Atlantic 1999 2 6511.800833 3185.28 3581.75 3410.75 +Children amalgexporti #x Mid Atlantic 1999 4 6854.405833 3541.62 3854.33 3938.42 +Men exportiimporto #x North Midwest 1999 3 7343.719167 4033.37 3984.13 3729.62 +Men amalgimporto #x Mid Atlantic 1999 3 6618.534167 3313.62 4044.36 4291.66 +Shoes exportiedu pack #x Mid Atlantic 1999 7 7416.141667 4121.24 4239.08 8658.42 +Women importoamalg #x Mid Atlantic 1999 6 6395.326667 3102.55 4234.22 3650.03 +Children importoexporti #x North Midwest 1999 3 6577.143333 3291.07 3773.61 2152.15 +Women edu packamalg #x NY Metro 1999 4 6027.880000 2750.96 3199.35 2575.81 +Music amalgscholar #x Mid Atlantic 1999 3 6662.669167 3386.25 1961.57 4799.18 +Men amalgimporto #x North Midwest 1999 6 7098.680833 3834.90 4923.53 4115.57 +Shoes importoedu pack #x Mid Atlantic 1999 3 6864.320833 3606.28 3104.48 3869.15 +Music exportischolar #x NY Metro 1999 6 7043.051667 3793.48 4135.21 5006.69 +Shoes edu packedu pack #x Mid Atlantic 1999 1 6711.753333 3473.10 15060.83 4085.86 +Men exportiimporto #x Mid Atlantic 1999 1 7419.459167 4188.88 16358.86 4366.77 +Women amalgamalg #x NY Metro 1999 2 6362.709167 3137.03 4180.91 3181.26 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q58.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q58.sql.out new file mode 100644 index 0000000000000..7e282de310b78 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q58.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAOEHBAAAA 4202.05 11.64402731132023188 3952.65 10.95293120074485419 3874.50 10.73637482126824727 4009.733333 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q59.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q59.sql.out new file mode 100644 index 0000000000000..020860f8baefd --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q59.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5271 0.74419775623719778195 0.74258389025810779226 0.18455097389041153754 0.11049123944133143765 2.01369701972914228390 0.74322255494522317957 0.71433787179231334416 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5272 0.84505990801516636507 1.64762915953700921988 0.38486496991736952802 0.68701158883293751293 1.56192545028377515216 1.16497445041784212487 0.46921185549145987093 +able AAAAAAAACAAAAAAA 5273 0.35130200745800939483 0.85485871077793239664 0.66980552712384851586 0.66252919917140464542 0.75558600483388711996 0.52111936828952912290 0.63215140490620731712 +able AAAAAAAACAAAAAAA 5273 0.35130200745800939483 0.85485871077793239664 0.66980552712384851586 0.66252919917140464542 0.75558600483388711996 0.52111936828952912290 0.63215140490620731712 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q60.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q60.sql.out new file mode 100644 index 0000000000000..632d983192f6f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q60.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAABAAAAA 6334.66 +AAAAAAAAAACCAAAA 5389.16 +AAAAAAAAAADBAAAA 11655.98 +AAAAAAAAAAHAAAAA 5388.00 +AAAAAAAAAAHDAAAA 12685.87 +AAAAAAAAAAIAAAAA 8916.30 +AAAAAAAAAAKBAAAA 2140.04 +AAAAAAAAAAMBAAAA 22048.17 +AAAAAAAAAAPBAAAA 13195.87 +AAAAAAAAABAEAAAA 1071.67 +AAAAAAAAABBCAAAA 7064.02 +AAAAAAAAABEAAAAA 13239.06 +AAAAAAAAABFBAAAA 1877.72 +AAAAAAAAABFEAAAA 699.95 +AAAAAAAAABGAAAAA 8169.00 +AAAAAAAAABJAAAAA 10752.87 +AAAAAAAAABLBAAAA 5962.43 +AAAAAAAAABLCAAAA 18131.46 +AAAAAAAAABMBAAAA 12049.00 +AAAAAAAAABMDAAAA 18947.93 +AAAAAAAAABNDAAAA 8479.29 +AAAAAAAAACDCAAAA 13466.87 +AAAAAAAAACEEAAAA 7634.65 +AAAAAAAAACJCAAAA 51774.63 +AAAAAAAAACKBAAAA 18766.24 +AAAAAAAAACLBAAAA 16002.89 +AAAAAAAAACLDAAAA 12302.68 +AAAAAAAAACOBAAAA 2707.13 +AAAAAAAAADABAAAA 6234.38 +AAAAAAAAADBDAAAA 27724.30 +AAAAAAAAADEAAAAA 2867.87 +AAAAAAAAADFAAAAA 10667.94 +AAAAAAAAADFDAAAA 9051.70 +AAAAAAAAADGBAAAA 6502.68 +AAAAAAAAADGCAAAA 1174.33 +AAAAAAAAADHDAAAA 18882.96 +AAAAAAAAADNBAAAA 11526.41 +AAAAAAAAADPBAAAA 1026.20 +AAAAAAAAAEADAAAA 3582.21 +AAAAAAAAAECBAAAA 12360.25 +AAAAAAAAAECCAAAA 20954.85 +AAAAAAAAAEDAAAAA 8537.71 +AAAAAAAAAEECAAAA 10513.50 +AAAAAAAAAEEDAAAA 2561.17 +AAAAAAAAAEGAAAAA 7836.96 +AAAAAAAAAEGDAAAA 638.36 +AAAAAAAAAEJBAAAA 11429.14 +AAAAAAAAAELBAAAA 6543.86 +AAAAAAAAAEMDAAAA 13797.32 +AAAAAAAAAEOBAAAA 6699.66 +AAAAAAAAAFAAAAAA 1979.04 +AAAAAAAAAFCBAAAA 6157.90 +AAAAAAAAAFDAAAAA 14494.27 +AAAAAAAAAFEBAAAA 11552.54 +AAAAAAAAAFGAAAAA 13218.28 +AAAAAAAAAFGCAAAA 10293.39 +AAAAAAAAAFGDAAAA 6078.36 +AAAAAAAAAFJDAAAA 21440.62 +AAAAAAAAAFLBAAAA 11714.63 +AAAAAAAAAFNCAAAA 17910.15 +AAAAAAAAAGACAAAA 12212.53 +AAAAAAAAAGAEAAAA 17187.86 +AAAAAAAAAGBAAAAA 2753.50 +AAAAAAAAAGDBAAAA 5244.60 +AAAAAAAAAGDEAAAA 466.32 +AAAAAAAAAGEAAAAA 2030.78 +AAAAAAAAAGEBAAAA 673.70 +AAAAAAAAAGEDAAAA 4861.80 +AAAAAAAAAGIAAAAA 20717.88 +AAAAAAAAAGKAAAAA 4481.54 +AAAAAAAAAGLAAAAA 259.22 +AAAAAAAAAGNDAAAA 9371.85 +AAAAAAAAAGOAAAAA 12624.61 +AAAAAAAAAGODAAAA 15071.99 +AAAAAAAAAHBDAAAA 6674.73 +AAAAAAAAAHCCAAAA 4327.67 +AAAAAAAAAHECAAAA 13309.93 +AAAAAAAAAHGAAAAA 8277.89 +AAAAAAAAAHIBAAAA 6574.61 +AAAAAAAAAHICAAAA 4810.18 +AAAAAAAAAHJDAAAA 9151.90 +AAAAAAAAAHKAAAAA 424.72 +AAAAAAAAAHKCAAAA 23702.30 +AAAAAAAAAHKDAAAA 8747.48 +AAAAAAAAAHMBAAAA 17707.03 +AAAAAAAAAHMDAAAA 5608.89 +AAAAAAAAAHNAAAAA 9929.30 +AAAAAAAAAHOCAAAA 21257.55 +AAAAAAAAAIAAAAAA 36659.16 +AAAAAAAAAICEAAAA 23564.63 +AAAAAAAAAIDCAAAA 7648.02 +AAAAAAAAAIEBAAAA 6151.13 +AAAAAAAAAIEEAAAA 1406.98 +AAAAAAAAAIFBAAAA 2088.56 +AAAAAAAAAIGAAAAA 2468.08 +AAAAAAAAAIGCAAAA 1834.76 +AAAAAAAAAIGDAAAA 558.00 +AAAAAAAAAIIBAAAA 5422.03 +AAAAAAAAAILDAAAA 4833.43 +AAAAAAAAAIOAAAAA 8317.81 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q61.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q61.sql.out new file mode 100644 index 0000000000000..e7ff87edde3d3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q61.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +2574995.94 5792384.50 44.4548517108972997220 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q62.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q62.sql.out new file mode 100644 index 0000000000000..879eb4c6355ac --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q62.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct120 days :bigint> +-- !query output +Just good amou EXPRESS site_0 270 248 280 287 0 +Just good amou EXPRESS site_1 290 327 288 262 0 +Just good amou EXPRESS site_2 314 295 311 299 0 +Just good amou EXPRESS site_3 309 300 299 267 0 +Just good amou EXPRESS site_4 224 250 297 273 0 +Just good amou LIBRARY site_0 217 192 210 225 0 +Just good amou LIBRARY site_1 163 207 172 205 0 +Just good amou LIBRARY site_2 176 204 208 190 0 +Just good amou LIBRARY site_3 222 220 240 240 0 +Just good amou LIBRARY site_4 254 230 241 263 0 +Just good amou NEXT DAY site_0 311 275 303 291 0 +Just good amou NEXT DAY site_1 291 307 284 289 0 +Just good amou NEXT DAY site_2 275 299 323 279 0 +Just good amou NEXT DAY site_3 260 290 242 248 0 +Just good amou NEXT DAY site_4 233 285 271 281 0 +Just good amou OVERNIGHT site_0 242 262 221 248 0 +Just good amou OVERNIGHT site_1 251 245 249 233 0 +Just good amou OVERNIGHT site_2 198 226 209 204 0 +Just good amou OVERNIGHT site_3 168 199 184 175 0 +Just good amou OVERNIGHT site_4 237 231 193 207 0 +Just good amou REGULAR site_0 140 236 154 190 0 +Just good amou REGULAR site_1 189 190 205 194 0 +Just good amou REGULAR site_2 234 251 264 235 0 +Just good amou REGULAR site_3 229 251 244 243 0 +Just good amou REGULAR site_4 198 209 188 216 0 +Just good amou TWO DAY site_0 256 258 218 235 0 +Just good amou TWO DAY site_1 217 226 195 198 0 +Just good amou TWO DAY site_2 176 151 186 202 0 +Just good amou TWO DAY site_3 221 225 236 190 0 +Just good amou TWO DAY site_4 239 240 219 252 0 +Matches produce EXPRESS site_0 330 277 239 314 0 +Matches produce EXPRESS site_1 296 336 288 307 0 +Matches produce EXPRESS site_2 262 307 273 305 0 +Matches produce EXPRESS site_3 284 253 253 256 0 +Matches produce EXPRESS site_4 260 309 252 296 0 +Matches produce LIBRARY site_0 188 181 179 170 0 +Matches produce LIBRARY site_1 189 211 193 181 0 +Matches produce LIBRARY site_2 253 230 207 248 0 +Matches produce LIBRARY site_3 225 230 217 214 0 +Matches produce LIBRARY site_4 220 218 208 213 0 +Matches produce NEXT DAY site_0 296 286 328 316 0 +Matches produce NEXT DAY site_1 303 303 307 281 0 +Matches produce NEXT DAY site_2 271 258 253 256 0 +Matches produce NEXT DAY site_3 265 263 290 296 0 +Matches produce NEXT DAY site_4 312 312 291 311 0 +Matches produce OVERNIGHT site_0 217 254 216 258 0 +Matches produce OVERNIGHT site_1 214 187 229 230 0 +Matches produce OVERNIGHT site_2 171 176 188 174 0 +Matches produce OVERNIGHT site_3 181 218 193 219 0 +Matches produce OVERNIGHT site_4 244 216 237 234 0 +Matches produce REGULAR site_0 205 216 204 229 0 +Matches produce REGULAR site_1 230 232 229 244 0 +Matches produce REGULAR site_2 225 250 209 254 0 +Matches produce REGULAR site_3 230 212 212 195 0 +Matches produce REGULAR site_4 207 167 180 172 0 +Matches produce TWO DAY site_0 228 195 222 198 0 +Matches produce TWO DAY site_1 169 154 170 146 0 +Matches produce TWO DAY site_2 197 215 201 210 0 +Matches produce TWO DAY site_3 247 236 234 237 0 +Matches produce TWO DAY site_4 242 259 245 247 0 +Operations EXPRESS site_0 270 284 268 286 0 +Operations EXPRESS site_1 274 251 245 275 0 +Operations EXPRESS site_2 287 337 296 310 0 +Operations EXPRESS site_3 349 307 322 286 0 +Operations EXPRESS site_4 284 287 285 271 0 +Operations LIBRARY site_0 266 227 229 237 0 +Operations LIBRARY site_1 201 228 190 200 0 +Operations LIBRARY site_2 177 170 182 171 0 +Operations LIBRARY site_3 227 217 214 226 0 +Operations LIBRARY site_4 268 258 242 242 0 +Operations NEXT DAY site_0 269 255 273 257 0 +Operations NEXT DAY site_1 292 295 258 291 0 +Operations NEXT DAY site_2 296 327 305 294 0 +Operations NEXT DAY site_3 274 286 322 289 0 +Operations NEXT DAY site_4 311 270 243 255 0 +Operations OVERNIGHT site_0 217 205 193 227 0 +Operations OVERNIGHT site_1 221 243 232 237 0 +Operations OVERNIGHT site_2 247 243 235 238 0 +Operations OVERNIGHT site_3 242 218 205 223 0 +Operations OVERNIGHT site_4 181 172 192 186 0 +Operations REGULAR site_0 190 211 206 195 0 +Operations REGULAR site_1 166 173 173 174 0 +Operations REGULAR site_2 198 219 215 226 0 +Operations REGULAR site_3 255 234 239 215 0 +Operations REGULAR site_4 246 231 266 208 0 +Operations TWO DAY site_0 247 232 246 240 0 +Operations TWO DAY site_1 244 240 259 234 0 +Operations TWO DAY site_2 210 214 200 190 0 +Operations TWO DAY site_3 166 181 174 184 0 +Operations TWO DAY site_4 194 197 224 211 0 +Selective, EXPRESS site_0 290 275 273 332 0 +Selective, EXPRESS site_1 303 319 311 299 0 +Selective, EXPRESS site_2 299 238 268 261 0 +Selective, EXPRESS site_3 262 273 281 300 0 +Selective, EXPRESS site_4 285 298 291 300 0 +Selective, LIBRARY site_0 215 204 202 211 0 +Selective, LIBRARY site_1 221 253 243 240 0 +Selective, LIBRARY site_2 218 261 226 234 0 +Selective, LIBRARY site_3 178 189 233 217 0 +Selective, LIBRARY site_4 159 186 194 199 0 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q63.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q63.sql.out new file mode 100644 index 0000000000000..4c5123f2cc90b --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q63.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +1 175.21 726.555833 +1 231.64 726.555833 +1 400.27 726.555833 +1 409.69 726.555833 +1 468.02 726.555833 +1 530.63 726.555833 +1 880.08 726.555833 +1 972.96 726.555833 +1 1028.83 726.555833 +1 1038.59 726.555833 +1 1820.16 726.555833 +2 392.91 1124.957500 +2 469.26 1124.957500 +2 654.97 1124.957500 +2 703.18 1124.957500 +2 726.27 1124.957500 +2 813.75 1124.957500 +2 819.59 1124.957500 +2 1248.04 1124.957500 +2 1253.65 1124.957500 +2 1690.71 1124.957500 +2 1961.13 1124.957500 +2 2766.03 1124.957500 +3 259.59 1110.663333 +3 530.95 1110.663333 +3 639.03 1110.663333 +3 727.70 1110.663333 +3 822.22 1110.663333 +3 853.58 1110.663333 +3 910.78 1110.663333 +3 1455.11 1110.663333 +3 1615.68 1110.663333 +3 1741.42 1110.663333 +3 2618.11 1110.663333 +4 465.88 1333.585833 +4 472.55 1333.585833 +4 685.93 1333.585833 +4 703.90 1333.585833 +4 773.55 1333.585833 +4 1535.08 1333.585833 +4 1900.09 1333.585833 +4 2557.73 1333.585833 +4 2842.07 1333.585833 +5 507.40 1378.418333 +5 604.37 1378.418333 +5 633.93 1378.418333 +5 830.17 1378.418333 +5 1013.51 1378.418333 +5 1093.85 1378.418333 +5 1223.10 1378.418333 +5 1735.77 1378.418333 +5 2024.74 1378.418333 +5 2065.17 1378.418333 +5 2319.65 1378.418333 +5 2489.36 1378.418333 +6 852.04 1879.384167 +6 975.86 1879.384167 +6 1082.92 1879.384167 +6 1121.29 1879.384167 +6 1133.12 1879.384167 +6 1558.44 1879.384167 +6 1581.26 1879.384167 +6 1654.62 1879.384167 +6 2408.41 1879.384167 +6 3334.37 1879.384167 +6 4855.76 1879.384167 +7 655.33 1433.785833 +7 691.77 1433.785833 +7 750.55 1433.785833 +7 862.94 1433.785833 +7 868.32 1433.785833 +7 992.39 1433.785833 +7 1209.78 1433.785833 +7 1694.72 1433.785833 +7 2081.47 1433.785833 +7 2257.98 1433.785833 +7 2383.35 1433.785833 +7 2756.83 1433.785833 +8 54.86 410.067500 +8 183.63 410.067500 +8 192.44 410.067500 +8 271.04 410.067500 +8 472.18 410.067500 +8 492.97 410.067500 +8 644.56 410.067500 +8 669.81 410.067500 +8 676.94 410.067500 +9 479.27 1306.207500 +9 572.00 1306.207500 +9 742.40 1306.207500 +9 759.92 1306.207500 +9 805.90 1306.207500 +9 993.12 1306.207500 +9 1030.85 1306.207500 +9 1764.83 1306.207500 +9 1826.50 1306.207500 +9 2065.07 1306.207500 +9 2158.76 1306.207500 +9 2475.87 1306.207500 +10 475.18 1442.284167 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q64.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q64.sql.out new file mode 100644 index 0000000000000..6caf086ed3028 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q64.sql.out @@ -0,0 +1,14 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +ablepricallyantiought ation 35709 71 River River Friendship 34536 NULL NULL Newport NULL 1999 1 22.60 38.87 0.00 17.09 26.31 0.00 2000 1 +ablepricallyantiought ation 35709 996 NULL Bridgeport 65817 752 Lakeview Lincoln Friendship 74536 1999 1 15.78 24.93 0.00 17.09 26.31 0.00 2000 1 +ablepricallyantiought bar 31904 128 East Franklin 19101 990 2nd Friendship 94536 1999 1 54.76 78.30 0.00 15.80 23.54 0.00 2000 1 +ationbarpri ation 35709 362 Central Ridge Oakland 69843 666 13th Ridge Shiloh 29275 1999 1 74.00 95.46 0.00 11.32 20.94 0.00 2000 1 +ationbarpri ese 31904 759 Elm Pine Belmont 20191 35 Madison Waterloo 31675 1999 1 12.92 22.22 0.00 24.15 36.70 0.00 2000 1 +ationbarpri ese 31904 759 Elm Pine Belmont 20191 35 Madison Waterloo 31675 1999 1 12.92 22.22 0.00 83.87 147.61 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 12.02 12.74 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 28.08 38.18 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 56.60 63.39 0.00 2000 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q65.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q65.sql.out new file mode 100644 index 0000000000000..16a7d8785e484 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q65.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +able Academic concerns help during a courses. Hard, working-class types concentrate in a costs. Commercial shares work directly additional, ordinary orders. Profi 29.50 3.57 2.92 amalgamalg #x +able Available ministers s 20.59 7.69 3.99 exportiimporto #x +able Before main women cover so rive 35.39 4.63 2.36 importoamalg #x +able Days will lift for a months. Public differences could like 30.44 1.78 0.99 importoexporti #x +able Fields expire perhaps available schools. Common machines allow later employees; lawyers will pull worldwide objectives. Hidden orders may turn well fo 19.20 88.16 54.65 importoimporto #x +able General, sure features drop for the time being integrated proposals. Occasionally eastern generations might not arrive today ever vast opportunities. Soft attacks used to sell 31.34 1.16 0.90 exportiexporti #x +able Lost, individual parents rally. Then various differences shall fill never private strong courts. Successfully global directors clean medical 2.94 8.89 5.95 edu packimporto #x +able Members say then authorities. Various wome 32.82 1.34 1.17 amalgamalgamalg #x +able National, round fields would not accomp 32.77 0.17 0.05 amalgnameless #x +able New measures shall pay under a agencies; comparatively heavy police shall beat similarly concepts. However japanese times cannot check like a police. Long, long-term auth 21.49 1.87 1.19 amalgnameless #x +able Only concerned times used to know however in the trees. Developers might not wear in the times. Studies see far variations. Calculations must not transport hardl 32.50 0.15 0.12 corpmaxi #x +able Only, foreign problems make in a women. Naturally very countries will beli 13.72 7.58 5.15 scholarunivamalg #x +able Perfect grants fight highly as great minutes. Severe, available millions like counties. Young legs cook however from a years. Early armed services reject yet with 23.87 4.31 2.84 scholarnameless #x +able Popular, ethical types must see human, central steps. Other options spare products; large, odd machines must fill clear, public users. Away local teachers make l 25.26 8.08 5.73 amalgexporti #x +able Privileges cut perhaps reasons. Ideas finish times. Women envy general programmes. Hands shall unveil never to a facilities; official proposals conform. Scot 18.45 7.52 4.73 univmaxi #x +able Probably suitable compan 21.47 2.39 1.29 importobrand #x +able Royal speeches take evil, front margins. For example hard events ought to go angles. Possible, foreign lakes shall not reconsider. Other honours hear momen 7.23 8.13 2.84 scholarnameless #x +able Rules offer. Important, italian goo 9.19 4.06 2.51 scholarmaxi #x +able Ships could not introduce as. Complete equations take different, european names. Respondents would help fine styles; really formal workers 32.96 2.55 2.04 edu packunivamalg #x +able Stores visit values. Others cannot hang around rather civil brothers. Direct systems go then free, other instructions. Difficult, top feet will al 33.93 13.91 6.39 exportinameless #x +able Systematic 12.54 2.98 2.17 amalgbrand #x +able Times concentrate religious forms. Soon social agents understand again on a officials. Miles w 25.36 5.16 2.58 importoexporti #x +ation Areas establish in 28.85 93.54 69.21 amalgunivamalg #x +ation Arrangements 26.15 0.27 0.20 edu packunivamalg #x +ation Boring areas used to print; companies delegate lines. Clients shall amount also then 18.54 3.84 1.84 importoexporti #x +ation Deliberately ordinary procedures will not pay by a months. Feet reach very s 34.44 9.43 5.46 univunivamalg #x +ation Ears might remind far charges. Pleased years discharge oth 24.58 2.73 1.22 edu packbrand #x +ation However old figures ask only good, large sources. Yet naked researchers shall deal to a women. Right, common miles describe there also prime bags. Readily significant shares 21.02 7.78 4.97 brandbrand #x +ation Just possible women say. Reasonably strong employers light almost degrees. Palestinian, smart rules help really visual 33.67 3.71 2.26 scholarbrand #x +ation Local years may not seek formally hard, interesting properties. Local values serve. Nevertheless private tales c 28.58 8.36 5.68 corpunivamalg #x +ation Much present elements regard previously glorious homes. Important, royal judges can ad 12.98 8.12 6.41 edu packscholar #x +ation Police succeed schools; supplies calculate far countries; new words move shares; officers must complete years. Asian things may bear warm things. Aw 29.31 6.66 2.26 univnameless #x +ation Respondents see also. Sure, american horses must go actually election 34.94 0.85 0.50 exportiamalg #x +ation Small, japanese rights will not think enough on the cars. Very fond hospitals may choose originally. Right, other businesses relish. As large decades include federal tears. Usual, important quar 21.23 3.63 1.81 amalgscholar #x +ation Technical, open seats used to become accordingly. Real, actual qualifications may not carry highly interesting others. Wide, sexual knees may stay expenses. Labour, american 30.78 0.61 0.51 exportiedu pack #x +ation Ultimately good sets could go short, early examinations. Things ought to know relatively. Linguistic, applicable children establish curiou 35.23 6.22 3.54 importoimporto #x +ation Unacceptable flowers should not give reasonable, ethnic governments. Employees shall complain 35.82 8.39 5.87 corpmaxi #x +bar Also coming years need with a faces; fresh poems judge for example already thick choices. Hands help individual, relevant samples. Together british fingers would not spe 23.68 2.15 0.98 edu packexporti #x +bar Bad, recent right 31.34 0.96 0.57 importoimporto #x +bar Cards should strike largely in a concessions. Still true signs might talk; essentially ro 29.21 55.23 48.05 importoamalg #x +bar Cases must not spend on 14.60 1.30 1.17 importoimporto #x +bar Cold, old days stem thereby difficult, nuclear men; likely contents shall threaten often outer years. All real or 3.53 9.08 3.35 edu packnameless #x +bar Conscious, solar ambitions support outside countries; warm facilities rise occupations. Appropriate columns grow. Availabl 26.55 3.35 2.84 importonameless #x +bar General, legal businesses should use completely expensive teachers. Linguistic friends vote problem 0.21 0.53 0.32 scholarbrand #x +bar High, official employees shall not start too left circumstances. Patients used to touch obviously popular, senior members. British, impossible theories make only. Young, international wo 28.68 4.85 3.63 edu packmaxi #x +bar However old hours ma 22.18 8.84 5.65 exportimaxi #x +bar Illegal technologies might distinguish that on a change 12.14 2.73 1.28 namelessmaxi #x +bar Never various elements play other rights 30.37 0.37 0.28 edu packscholar #x +bar Newspapers would ensure certainly short inadequate problems. Bedrooms would argue however halfway bad coun 29.16 2.78 1.69 scholarcorp #x +bar Only groups will not prove united, furious results. Papers shall think in a opportunities; techniques decide that. American, initial forces might mean previous books. That delighted arts give so dimen 28.98 2.78 1.64 exportiimporto #x +bar Open prod 11.05 2.74 1.53 edu packunivamalg #x +bar Presidential, mild tests justify yesterday unusual points. Notable individuals can set only external trousers. Here french letters may photogra 27.05 2.44 2.17 importoimporto #x +bar Prices acquire more out of a christians. Efficiently local prices 20.48 2.11 0.78 exportimaxi #x +bar Rare, full workers stay plant 8.61 0.55 0.34 exportiimporto #x +bar Social pieces become; reservations rescue probably old hopes. Different, high records buy just general centuries. Recently industrial relationships cannot 16.63 9.15 5.03 importoedu pack #x +bar Though private depths accomplis 12.68 0.58 0.23 amalgamalgamalg #x +bar Thus certain stars appear totally even local guests. Urban friends might not take properly various vehicles 35.61 4.55 2.09 amalgunivamalg #x +bar Traditional legs pull at least better difficult circumstances; other, inner clients step burning arms; able, numerous weapons keep li 3.50 45.72 20.11 amalgscholar #x +bar Unknown topics ought to answer far present pictures. Estimated considerations might meet 32.53 2.55 1.32 exportiamalg #x +bar Widely splendid others deprive only. Different, main soldiers discover then other periods. Too main birds must change public, terrible houses. Different, armed females may foster; science 22.86 4.26 3.36 amalgunivamalg #x +eing Formerly central designers must not save. Scottish, small horses elicit men. British, fine companies pay little taxes; here pure lakes should benefit however small top countr 33.81 0.51 0.31 exportiamalg #x +eing Forms shall involve then normal bodies. Left words may find 11.97 2.02 0.64 edu packexporti #x +eing Front elections ensure to a adults; valuable moments decide in a aspects. Marked books stand rooms. Expectations ought 37.36 59.51 43.44 amalgimporto #x +eing Gradual volunteers keep bc months. Calls get pleasantly questions. Repre 12.82 1.61 1.07 edu packexporti #x +eing Gradually new flowers support suddenly. Left, light errors ought to steal other memories. Periods should not say never to a nurses; 31.57 2.05 1.12 edu packamalg #x +eing Great, possible children used to 11.87 4.00 2.60 namelessnameless #x +eing Neat, desirable words make especially gradu 25.33 7.11 3.76 amalgnameless #x +eing Never top observations spend appropriate, common states. Homes make. There available hospitals will appreciate away upon a years. Roots hang 31.15 2.07 0.84 edu packnameless #x +eing Often 21.89 7.85 4.16 amalgscholar #x +eing Poor years produce questions. Marine events ensure inner systems. Individuals could kill to a managers. Drugs should not authorise thankfully traditional, strong holders. Just amazing injuri 19.66 4.11 3.16 importoimporto #x +eing Sensibly foreign parties must not suffer well indian personal students. About private cattle handle th 4.79 2.42 1.01 amalgscholar #x +eing Silver, political interviews might know in common families. Far possible houses shall insist in a places. Whole, political gardens would adopt eggs. Others might live even offi 33.77 6.13 2.88 namelessmaxi #x +eing Social areas undergo actually within a plants. More communist days would play on a books. Later educational policies shoul 30.97 5.52 2.92 exportiunivamalg #x +eing Soon artificial notions think no longer lights; clearly late members could not trace good countries. Cultures can proceed away wealthy 0.98 2.38 1.68 exportibrand #x +ese Additional, interior police provide words. Different, long qualities answer really concerns; then other words state dry, political services. Awfully di 15.35 9.78 7.53 corpbrand #x +ese Concer 32.51 9.66 8.11 amalgamalg #x +ese Constitutional, good pupils might not begin below level devices. External savings fit hardly. Parents shall dry. Actually literary companies improve a 17.22 4.22 2.40 scholarunivamalg #x +ese Earnings feel possibilities. Single, poor problems make full, sho 29.18 2.75 1.62 maximaxi #x +ese Financial, likely artists assume now c 5.10 5.63 2.42 importoamalgamalg #x +ese Forces enter efficient things. S 23.42 2.64 0.81 amalgamalgamalg #x +ese Forward new lights should not meet once yesterday national buildings. Natural, australian eyes may not fetch progressively unfair 29.40 7.87 5.66 amalgimporto #x +ese Fresh respondents would not encourage in a years. 26.81 3.90 1.67 importoimporto #x +ese Goals ought to strengthen. Early industries would take. Early men could hear then. Difficult, new machines endorse 9.97 1.24 0.57 importoamalg #x +ese Identical solicitors must maintain sources. Factors take already unusual minutes. Just various sales sell agricultural, long states. 27.99 3.77 1.16 maxinameless #x +ese In order turkish meanings should involve nevertheless to a inches. Common, free colleagues may know other, safe services. So sure plates might go hidden, formidable powers. Domes 33.33 1.12 0.35 namelesscorp #x +ese Just associated missiles could 20.43 1.23 0.46 edu packedu pack #x +ese Meanwhile single companies shall go either in a statements. Vo 9.56 7.96 3.42 corpbrand #x +ese Models feel figures. Homes get still at the positions. Political, other makers will make there in the servants; necessary, technical markets should not cope also; warm, 32.66 6.16 4.49 univbrand #x +ese New interests must turn largely. High, essential females mark. Gradual police would not exercise old, national 23.00 6.87 2.33 edu packcorp #x +ese Parliamentary pieces shine never tragic patterns. Great human eyes would not get groups. Plant 9.58 6.03 4.82 scholarmaxi #x +ese Photographs get everywhere merely 13.94 3.75 2.96 exportiamalg #x +ese Plates shall think; new, economic pupils collect entirely. Really powerful books develop yet girls. Best unlik 33.09 3.44 2.99 namelessbrand #x +ese Proposals should involve more soviet, young islands. Little resources try even books. Fundamental systems end recent, total provisions. Working-class matte 30.46 5.15 3.60 exportischolar #x +ese Public operations need wonderfully improved routes; days may not admit for a circles. Able, wise girls lay later. Authorities know really reasons. Scottish accountants take customs. 2.20 1.40 0.99 amalgimporto #x +ese Sharply bright systems used to want. Other projects should benefit. Common parts use 35.01 1.16 0.39 importoexporti #x +ese Short, known programmes reject quite documents. Really interna 7.78 36.94 22.90 univamalgamalg #x +ese Small years could spend soon 31.85 0.55 0.39 importoscholar #x +ese Special gates decide mutually. Current, appropriate terms feel thus better royal arms. Children starve; likely girls make yesterday local workers. 26.14 2.93 1.11 exportischolar #x +ese There round authorities will show in a seasons. Other, far firms miss very mad, private lips. Powerful, m 16.77 5.93 4.44 importoamalg #x +ese Women get also chairs. Full, integrated paintings sit 1.47 6.34 2.15 scholarnameless #x diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q66.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q66.sql.out new file mode 100644 index 0000000000000..3287cb908761f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q66.sql.out @@ -0,0 +1,10 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Just good amou 933435 Midway Williamson County TN United States DHL,BARIAN 2001 9246702.65 4973596.11 8243830.24 11536095.04 8848441.04 6589907.46 14946163.85 24260830.90 23156311.42 21897174.69 33046376.63 24591398.12 9.906102353136 5.328272573880 8.831713231238 12.358755606978 9.479439961004 7.059846116762 16.012002817550 25.990916239481 24.807631404437 23.458703273394 35.402975708003 26.345056827738 29452942.94 19052763.82 30303577.65 28563648.00 19179558.75 25111623.08 26955891.11 62345171.47 55967232.63 60120132.61 94483100.18 85104727.47 +Matches produce 198821 Midway Williamson County TN United States DHL,BARIAN 2001 8171037.69 7444707.30 8904945.91 8019953.83 13081755.25 8229030.48 11805829.83 25145588.96 15707862.93 18731722.90 27837144.77 33176280.56 41.097457964702 37.444270474447 44.788759285990 40.337559060662 65.796647486935 41.389141388486 59.379189471937 126.473506118569 79.005049416309 94.214006065758 140.011089220958 166.865072401809 25824877.19 17086383.78 24620145.44 28759520.46 24988081.00 21709799.17 25386565.52 54025360.94 54325725.26 59965431.57 75829271.22 85511883.03 +Operations 500020 Fairview Williamson County TN United States DHL,BARIAN 2001 10409399.91 16141530.81 8148742.71 6680962.19 11728095.96 9767093.38 11458408.56 20106135.67 25314120.67 22893962.36 35876332.21 28738513.96 20.817967101315 32.281770349186 16.296833546658 13.361389924403 23.455253709851 19.533405423783 22.915900483980 40.210662913483 50.626216291349 45.786093276269 71.749794428223 57.474728930843 18813981.10 29928354.06 20237634.01 22392433.57 25938775.04 28927292.16 25641922.14 60075246.63 49668486.97 68646750.13 88585464.12 75478344.38 +Selective, 720621 Fairview Williamson County TN United States DHL,BARIAN 2001 10540885.44 8098669.29 9370217.14 9753408.96 9006503.21 7725824.55 12918857.72 27286339.36 17315063.55 22573845.28 39002921.92 33502638.23 14.627502445808 11.238458621106 13.002975405935 13.534727630752 12.498252493335 10.721064956475 17.927395565769 37.865034962900 24.027975246350 31.325544606665 54.124042901886 46.491343202599 27419902.20 22882381.71 25735954.25 18682264.05 24373316.64 27192350.35 33019111.44 54462656.98 50994470.37 58509681.21 88329849.81 100833156.77 +Significantly 200313 Fairview Williamson County TN United States DHL,BARIAN 2001 9802846.67 12625248.69 6602477.32 11033269.92 6082390.20 15618748.55 5368995.93 19287503.44 21876336.16 23132474.86 35201297.49 21425392.36 48.937645934113 63.027605247787 32.960802943393 55.080149166554 30.364430666008 77.971717012875 26.803032903506 96.286828313689 109.210765951287 115.481645524754 175.731467703044 106.959570072836 23574229.26 27163084.88 24006453.37 23356164.02 19405795.41 35795815.91 25006450.52 58626312.30 57258316.63 60471338.55 83474267.08 73080652.01 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q67.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q67.sql.out new file mode 100644 index 0000000000000..dbb955dcae502 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q67.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL 672624.93 5 +NULL NULL NULL NULL NULL NULL NULL NULL 1328761.89 4 +NULL NULL NULL NULL NULL NULL NULL NULL 1867833.60 3 +NULL NULL NULL NULL NULL NULL NULL NULL 2951322.99 2 +NULL NULL NULL NULL NULL NULL NULL NULL 1012102066.67 1 +NULL NULL NULL NULL 2000 NULL NULL NULL 672624.93 5 +NULL NULL NULL NULL 2000 1 NULL NULL 109354.01 39 +NULL NULL NULL NULL 2000 1 1 NULL 56696.62 88 +NULL NULL NULL NULL 2000 2 NULL NULL 117554.43 24 +NULL NULL NULL NULL 2000 2 6 NULL 50119.54 93 +NULL NULL NULL NULL 2000 3 NULL NULL 147622.32 12 +NULL NULL NULL NULL 2000 3 8 NULL 64629.56 85 +NULL NULL NULL NULL 2000 3 9 NULL 59755.91 86 +NULL NULL NULL NULL 2000 4 NULL NULL 298094.17 7 +NULL NULL NULL NULL 2000 4 10 NULL 96651.33 58 +NULL NULL NULL NULL 2000 4 10 AAAAAAAAKAAAAAAA 41992.37 100 +NULL NULL NULL NULL 2000 4 11 NULL 110100.01 35 +NULL NULL NULL NULL 2000 4 12 NULL 91342.83 68 +NULL NULL NULL ationesebareing NULL NULL NULL NULL 136570.22 16 +NULL NULL NULL ationesebareing 2000 NULL NULL NULL 136570.22 16 +NULL NULL NULL ationesebareing 2000 4 NULL NULL 54262.81 90 +NULL NULL NULL eseationablebarought NULL NULL NULL NULL 106226.70 44 +NULL NULL NULL eseationablebarought 2000 NULL NULL NULL 106226.70 44 +NULL NULL NULL eseationablebarought 2000 4 NULL NULL 46143.77 96 +NULL NULL NULL n stought NULL NULL NULL NULL 90518.72 69 +NULL NULL NULL n stought 2000 NULL NULL NULL 90518.72 69 +NULL NULL NULL oughtableantiable NULL NULL NULL NULL 96782.30 52 +NULL NULL NULL oughtableantiable 2000 NULL NULL NULL 96782.30 52 +NULL NULL NULL oughtableantiable 2000 4 NULL NULL 49543.67 94 +NULL NULL NULL oughtablen stationought NULL NULL NULL NULL 105616.28 46 +NULL NULL NULL oughtablen stationought 2000 NULL NULL NULL 105616.28 46 +NULL NULL NULL oughtablen stationought 2000 4 NULL NULL 66351.81 84 +NULL NULL NULL oughteseoughtation NULL NULL NULL NULL 120422.74 22 +NULL NULL NULL oughteseoughtation 2000 NULL NULL NULL 120422.74 22 +NULL NULL NULL oughteseoughtation 2000 3 NULL NULL 51575.65 92 +NULL NULL NULL oughteseoughtation 2000 4 NULL NULL 53356.26 91 +NULL NULL brandcorp #x NULL NULL NULL NULL NULL 110558.31 32 +NULL NULL brandcorp #x esepriantieing NULL NULL NULL NULL 110558.31 32 +NULL NULL brandcorp #x esepriantieing 2000 NULL NULL NULL 110558.31 32 +NULL NULL brandcorp #x esepriantieing 2000 4 NULL NULL 57539.32 87 +NULL NULL corpamalgamalg #x NULL NULL NULL NULL NULL 178041.47 9 +NULL NULL corpamalgamalg #x callyationationantiought NULL NULL NULL NULL 178041.47 9 +NULL NULL corpamalgamalg #x callyationationantiought 2000 NULL NULL NULL 178041.47 9 +NULL NULL corpamalgamalg #x callyationationantiought 2000 4 NULL NULL 92734.16 59 +NULL NULL corpmaxi #x NULL NULL NULL NULL NULL 140599.56 13 +NULL NULL corpmaxi #x callyeingpri NULL NULL NULL NULL 140599.56 13 +NULL NULL corpmaxi #x callyeingpri 2000 NULL NULL NULL 140599.56 13 +NULL NULL corpmaxi #x callyeingpri 2000 4 NULL NULL 68417.63 83 +NULL NULL corpmaxi #x callyeingpri 2000 4 11 NULL 44322.49 99 +NULL NULL importoamalg #x NULL NULL NULL NULL NULL 109872.37 36 +NULL NULL importoamalg #x NULL NULL NULL NULL NULL 109872.37 36 +NULL NULL importoamalg #x NULL 2000 NULL NULL NULL 109872.37 36 +NULL accessories NULL NULL NULL NULL NULL NULL 102003.04 48 +NULL accessories NULL NULL NULL NULL NULL NULL 102003.04 48 +NULL accessories NULL prin stn stoughtought NULL NULL NULL NULL 102003.04 48 +NULL accessories NULL prin stn stoughtought 2000 NULL NULL NULL 102003.04 48 +NULL accessories NULL prin stn stoughtought 2000 4 NULL NULL 56122.71 89 +NULL athletic NULL NULL NULL NULL NULL NULL 72722.46 79 +NULL athletic NULL NULL NULL NULL NULL NULL 72722.46 79 +NULL athletic NULL NULL NULL NULL NULL NULL 72722.46 79 +NULL athletic NULL NULL 2000 NULL NULL NULL 72722.46 79 +NULL baseball NULL NULL NULL NULL NULL NULL 96654.08 54 +NULL baseball NULL NULL NULL NULL NULL NULL 96654.08 54 +NULL baseball NULL NULL NULL NULL NULL NULL 96654.08 54 +NULL baseball NULL NULL 2000 NULL NULL NULL 96654.08 54 +NULL baseball NULL NULL 2000 3 NULL NULL 44473.92 98 +NULL country NULL NULL NULL NULL NULL NULL 114628.36 25 +NULL country importoscholar #x NULL NULL NULL NULL NULL 114628.36 25 +NULL country importoscholar #x NULL NULL NULL NULL NULL 114628.36 25 +NULL country importoscholar #x NULL 2000 NULL NULL NULL 114628.36 25 +NULL country importoscholar #x NULL 2000 4 NULL NULL 44753.51 97 +NULL dresses NULL NULL NULL NULL NULL NULL 77687.53 72 +NULL dresses NULL NULL NULL NULL NULL NULL 192205.73 8 +NULL dresses NULL oughteingantieing NULL NULL NULL NULL 77687.53 72 +NULL dresses NULL oughteingantieing 2000 NULL NULL NULL 77687.53 72 +NULL dresses amalgamalg #x NULL NULL NULL NULL NULL 114518.20 29 +NULL dresses amalgamalg #x NULL NULL NULL NULL NULL 114518.20 29 +NULL dresses amalgamalg #x NULL 2000 NULL NULL NULL 114518.20 29 +NULL dresses amalgamalg #x NULL 2000 4 NULL NULL 49313.59 95 +NULL guns NULL NULL NULL NULL NULL NULL 92681.88 60 +NULL guns NULL NULL NULL NULL NULL NULL 92681.88 60 +NULL guns NULL NULL NULL NULL NULL NULL 92681.88 60 +NULL guns NULL NULL 2000 NULL NULL NULL 92681.88 60 +NULL infants NULL NULL NULL NULL NULL NULL 129135.94 18 +NULL infants importoexporti #x NULL NULL NULL NULL NULL 129135.94 18 +NULL infants importoexporti #x antiationeseese NULL NULL NULL NULL 129135.94 18 +NULL infants importoexporti #x antiationeseese 2000 NULL NULL NULL 129135.94 18 +NULL infants importoexporti #x antiationeseese 2000 4 NULL NULL 80824.80 71 +NULL lighting NULL NULL NULL NULL NULL NULL 107477.64 40 +NULL lighting NULL NULL NULL NULL NULL NULL 107477.64 40 +NULL lighting NULL NULL NULL NULL NULL NULL 107477.64 40 +NULL lighting NULL NULL 2000 NULL NULL NULL 107477.64 40 +NULL mens NULL NULL NULL NULL NULL NULL 91415.81 64 +NULL mens importoedu pack #x NULL NULL NULL NULL NULL 91415.81 64 +NULL mens importoedu pack #x oughtablen steseought NULL NULL NULL NULL 91415.81 64 +NULL mens importoedu pack #x oughtablen steseought 2000 NULL NULL NULL 91415.81 64 +NULL shirts NULL NULL NULL NULL NULL NULL 72876.08 75 +NULL shirts NULL NULL NULL NULL NULL NULL 72876.08 75 +NULL shirts NULL oughtcallyeseantiought NULL NULL NULL NULL 72876.08 75 +NULL shirts NULL oughtcallyeseantiought 2000 NULL NULL NULL 72876.08 75 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q68.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q68.sql.out new file mode 100644 index 0000000000000..11a564346ec52 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q68.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL Salem Lakeside 2203 23464.72 675.04 46100.94 +NULL Ira Riverdale Oak Ridge 10508 32911.54 938.79 59812.32 +NULL Rebecca Belmont Marion 13206 14180.52 1072.25 23887.85 +NULL NULL Pleasant Valley Lincoln 20829 16340.86 289.50 34809.84 +NULL Joel Oak Hill Clifton 21390 19561.57 939.54 46319.37 +NULL Max Stringtown Pine Grove 22049 20059.77 974.94 38268.07 +NULL NULL Bridgeport Riverside 23264 10699.95 496.36 29285.70 +NULL NULL White Oak Pleasant Hill 24860 16258.45 733.62 52186.60 +NULL Rebecca Union Hill Farmington 26031 7636.54 515.78 13443.39 +NULL Alecia Jamestown Oakdale 30174 29125.41 1118.58 69857.70 +NULL Raymond Bunker Hill Clifton 30655 27237.08 1237.59 51322.22 +NULL NULL Pleasant Valley Riley 30915 25079.14 998.46 35200.27 +NULL David Glendale Greenville 33716 15434.87 1012.60 30661.71 +NULL NULL New Hope Farmington 38760 29150.85 647.52 50433.60 +NULL Norma Hamilton Macedonia 41929 35397.41 1268.87 55360.80 +NULL NULL Wildwood Allison 45786 10448.67 492.27 32424.46 +NULL NULL Greenville Spring Hill 47915 26747.77 512.05 57910.19 +NULL Jerome Galena Georgetown 50559 33831.95 890.50 57970.13 +NULL NULL Woodlawn Springfield 59625 22470.48 828.96 54458.90 +NULL NULL Hillcrest Cordova 64306 10534.53 295.14 15380.96 +NULL John Stringtown Arlington 64522 21495.01 783.81 45067.59 +NULL NULL Gladstone Summit 66488 20881.36 605.92 32362.22 +NULL NULL Hopewell Riverdale 67278 24471.59 1549.17 48406.81 +NULL Russel Red Hill Woodland 82061 17750.10 1025.09 40565.68 +NULL NULL Mount Olive Maple Grove 82183 16184.99 573.72 44448.96 +NULL NULL Shady Grove Springdale 91775 17103.31 911.07 35127.63 +NULL NULL Indian Village Golden 98530 18842.65 1005.52 40742.92 +NULL NULL Greenfield Farmington 98985 13264.63 589.99 20576.04 +NULL Virginia Five Forks Oakland 106543 24859.75 1006.88 39693.34 +NULL NULL Scottsville Greenfield 107236 14638.50 802.00 35356.18 +NULL Virginia Oak Hill Amherst 109641 7667.92 271.98 30755.88 +NULL NULL Midway New Hope 113272 50295.57 2963.67 78595.37 +NULL William Valley View Providence 115559 29947.61 673.81 50883.54 +NULL NULL Newport Indian Village 120966 15268.90 721.76 27022.81 +NULL Robert Cedar Grove Riverside 122552 20398.86 1049.35 44761.40 +NULL NULL Jamestown Spring Valley 125125 31795.01 1364.75 52740.75 +NULL NULL Liberty Oakwood 128427 41495.05 2399.87 77105.28 +NULL Brandon Arlington Woodland 135601 43789.02 1493.45 66479.43 +NULL Juan Mount Pleasant Greenwood 138846 12326.46 472.96 25735.00 +NULL NULL Shirley Fairview 143290 19808.13 596.98 34256.82 +NULL NULL White Oak Ferguson 146537 5235.24 246.72 11343.46 +NULL Luther Greenville Hillcrest 149654 27079.37 939.22 41067.50 +NULL Michelle Fairview Hopewell 149783 29018.58 1426.60 48040.63 +NULL Angelica Barnes Plainview 151659 21082.45 306.14 40857.65 +NULL Donna Centerville Friendship 152588 20862.64 969.20 46304.16 +NULL Brian Oak Ridge Woodville 152862 22493.16 1611.93 43153.08 +NULL NULL Stringtown Liberty 158006 16267.97 450.16 25629.75 +NULL Daniel Globe Spring Valley 161922 15092.01 847.21 42715.60 +NULL Marlene Georgetown Oakland 163799 42790.16 1130.51 77449.21 +NULL Dean Lakeview New Hope 167847 24431.20 1333.52 38658.46 +NULL NULL Greenfield Hopewell 168370 13878.80 617.56 31230.84 +NULL Robert Four Points Farmington 176911 14828.21 674.85 32517.24 +NULL Betty Lakewood Sulphur Springs 178057 15008.49 693.61 25864.62 +NULL NULL Harmony Spring Valley 178332 10288.94 631.05 35516.38 +NULL NULL Enterprise Bunker Hill 182211 14072.57 405.42 33771.00 +NULL NULL Salem Valley View 187904 15702.34 439.59 41237.94 +NULL Faye Walnut Grove Oakland 189770 18527.90 1113.75 39987.40 +NULL Debra Green Acres Woodlawn 192040 9053.50 169.31 13181.96 +NULL NULL Salem Antioch 192673 5412.31 119.61 12746.52 +NULL NULL Mount Olive Mount Pleasant 197746 32038.93 1017.39 87608.72 +NULL Nancy Shady Grove Hillcrest 201415 14985.28 328.13 21957.26 +NULL Scott Hillcrest Louisville 203119 8629.88 329.70 14712.89 +NULL NULL Plainview Oak Grove 203425 33448.32 1632.67 62674.71 +NULL NULL Rocky Hill Providence 211665 15060.41 598.75 29385.31 +NULL Tonya Buena Vista Bunker Hill 213228 46912.33 2039.49 81223.03 +NULL Troy Plainview Ashland 216821 45620.34 2295.90 74013.12 +NULL Sandra Unionville Hopewell 221077 8341.60 527.66 17817.11 +NULL Robert Summit Five Points 224508 15607.69 500.96 37984.93 +NULL Rebecca Union Hill Mountain View 225489 35322.48 1859.03 58568.59 +NULL Jack Sunnyside Lakeview 231229 16186.51 577.86 30596.37 +NULL NULL Bethel Sunnyside 233081 13271.69 468.80 32129.60 +NULL Alice Oak Grove Marion 233656 30278.85 1490.14 65326.68 +NULL Shayne Riverview Crossroads 234750 20554.66 737.10 52975.61 +NULL Kevin Buena Vista Clifton 239837 28589.10 999.94 48962.74 +Aaron Nick Harmony Edgewood 73734 29649.48 959.10 56974.10 +Abney Janice Mount Zion Enterprise 27585 16154.82 636.94 37751.16 +Abraham Gerald Pleasant Grove Midway 1779 8661.58 330.60 14811.87 +Abrams Alma Fairfield Macedonia 33078 28262.54 808.09 48263.03 +Adame Brian Marion Forest Hills 142280 15398.91 717.67 52277.17 +Adams Adam The Meadows Glenwood 35054 33420.92 1477.44 37326.99 +Adams Paulette Tremont Lakeside 82644 25822.05 365.13 46273.62 +Adams Edwin Plainview Greenwood 108138 23483.91 982.40 35181.82 +Adams Nichole Edgewood Friendship 171894 20821.96 1333.70 48375.06 +Adams NULL Oakland Oak Ridge 173206 18700.18 1085.29 29361.60 +Adams Nichole Edgewood Green Acres 180862 17519.04 657.52 33555.87 +Adams Donna Plainview Liberty 219849 18150.78 448.49 22444.41 +Aguilar Jeannine Unionville Jackson 152737 34732.21 1299.63 56844.65 +Ahmed Jeffrey Stewart Edgewood 202936 27333.88 948.83 39990.68 +Albert Sally Nottingham Mount Zion 7000 11496.97 305.77 28524.27 +Albrecht Bob Marion Pleasant Hill 60194 23528.46 1101.20 43551.97 +Alcorn Jeffery Cedar Grove Salem 46467 17634.79 785.53 57718.43 +Alderman Melanie Edgewood Mount Zion 14230 28157.47 270.85 48818.21 +Aldridge Daniel Berlin Wilson 114445 15440.33 552.85 22637.40 +Alger Beverly Allentown Riverview 173449 24541.75 797.49 48511.28 +Allen Brittany Lincoln Hillcrest 10696 17579.89 595.90 46637.81 +Allen Yvette Providence Flatwoods 73587 8938.13 561.96 27604.45 +Allen Lori Hamilton Greenwood 138690 16085.88 579.59 35840.88 +Allison Anya Union Ashland 79895 9453.18 499.78 22112.21 +Alvarez Terrence Concord Spring Valley 32211 13206.58 470.82 33690.82 +Alvarez Marie Walnut Grove Red Hill 206011 25535.02 1344.58 63224.42 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q69.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q69.sql.out new file mode 100644 index 0000000000000..bd90d440c523c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q69.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +F D 2 yr Degree 1 500 1 Low Risk 1 +F D 2 yr Degree 1 3000 1 High Risk 1 +F D 2 yr Degree 1 4500 1 Good 1 +F D 2 yr Degree 1 5000 1 Low Risk 1 +F D 2 yr Degree 2 6000 2 High Risk 2 +F D 2 yr Degree 1 6500 1 Good 1 +F D 2 yr Degree 1 7000 1 High Risk 1 +F D 2 yr Degree 1 8500 1 High Risk 1 +F D 2 yr Degree 1 8500 1 Low Risk 1 +F D 4 yr Degree 1 1000 1 High Risk 1 +F D 4 yr Degree 1 2500 1 Good 1 +F D 4 yr Degree 1 3000 1 Low Risk 1 +F D 4 yr Degree 1 3500 1 High Risk 1 +F D 4 yr Degree 1 3500 1 Unknown 1 +F D 4 yr Degree 1 4000 1 Unknown 1 +F D 4 yr Degree 1 4500 1 Unknown 1 +F D 4 yr Degree 1 6500 1 High Risk 1 +F D 4 yr Degree 1 7500 1 High Risk 1 +F D 4 yr Degree 1 8500 1 Low Risk 1 +F D 4 yr Degree 1 9000 1 Low Risk 1 +F D 4 yr Degree 1 9000 1 Unknown 1 +F D 4 yr Degree 1 10000 1 High Risk 1 +F D Advanced Degree 2 1000 2 Unknown 2 +F D Advanced Degree 1 1500 1 High Risk 1 +F D Advanced Degree 1 1500 1 Unknown 1 +F D Advanced Degree 1 3000 1 Good 1 +F D Advanced Degree 1 3000 1 Unknown 1 +F D Advanced Degree 1 6000 1 Low Risk 1 +F D Advanced Degree 1 8000 1 Unknown 1 +F D College 1 1000 1 Low Risk 1 +F D College 1 2000 1 High Risk 1 +F D College 3 2500 3 High Risk 3 +F D College 1 4000 1 High Risk 1 +F D College 1 5500 1 High Risk 1 +F D College 2 7500 2 Unknown 2 +F D College 1 8000 1 Good 1 +F D College 1 9000 1 Unknown 1 +F D Primary 1 500 1 Good 1 +F D Primary 1 1000 1 Unknown 1 +F D Primary 1 1500 1 Good 1 +F D Primary 1 2000 1 High Risk 1 +F D Primary 2 2000 2 Unknown 2 +F D Primary 1 2500 1 Unknown 1 +F D Primary 1 4000 1 Low Risk 1 +F D Primary 1 5000 1 High Risk 1 +F D Primary 1 6000 1 Unknown 1 +F D Primary 1 6500 1 High Risk 1 +F D Primary 1 7500 1 High Risk 1 +F D Primary 1 7500 1 Unknown 1 +F D Primary 1 8000 1 Low Risk 1 +F D Primary 2 9500 2 Low Risk 2 +F D Secondary 1 1500 1 High Risk 1 +F D Secondary 1 2000 1 Unknown 1 +F D Secondary 1 2500 1 High Risk 1 +F D Secondary 2 4000 2 High Risk 2 +F D Secondary 1 5000 1 Low Risk 1 +F D Secondary 1 6000 1 High Risk 1 +F D Secondary 1 10000 1 Low Risk 1 +F D Unknown 1 2000 1 High Risk 1 +F D Unknown 1 5000 1 Low Risk 1 +F D Unknown 1 6500 1 Good 1 +F D Unknown 1 8000 1 Unknown 1 +F D Unknown 1 9000 1 High Risk 1 +F M 2 yr Degree 1 500 1 High Risk 1 +F M 2 yr Degree 1 4000 1 Unknown 1 +F M 2 yr Degree 1 4500 1 Low Risk 1 +F M 2 yr Degree 1 5000 1 Unknown 1 +F M 2 yr Degree 1 5500 1 Low Risk 1 +F M 2 yr Degree 1 9000 1 Low Risk 1 +F M 2 yr Degree 1 10000 1 Low Risk 1 +F M 4 yr Degree 1 500 1 High Risk 1 +F M 4 yr Degree 1 1000 1 Good 1 +F M 4 yr Degree 1 2500 1 Good 1 +F M 4 yr Degree 1 3500 1 Low Risk 1 +F M 4 yr Degree 1 4500 1 Good 1 +F M 4 yr Degree 1 7000 1 Good 1 +F M 4 yr Degree 1 7500 1 Good 1 +F M 4 yr Degree 1 7500 1 High Risk 1 +F M 4 yr Degree 1 8000 1 Unknown 1 +F M 4 yr Degree 1 8500 1 High Risk 1 +F M 4 yr Degree 1 9000 1 Unknown 1 +F M 4 yr Degree 1 10000 1 Unknown 1 +F M Advanced Degree 1 3000 1 Unknown 1 +F M Advanced Degree 1 4000 1 Low Risk 1 +F M Advanced Degree 1 4500 1 Unknown 1 +F M Advanced Degree 1 6000 1 Unknown 1 +F M Advanced Degree 1 6500 1 High Risk 1 +F M Advanced Degree 1 8500 1 Good 1 +F M Advanced Degree 1 8500 1 High Risk 1 +F M College 1 1500 1 Unknown 1 +F M College 1 4500 1 Low Risk 1 +F M College 1 7000 1 Unknown 1 +F M College 1 8500 1 High Risk 1 +F M College 1 8500 1 Low Risk 1 +F M College 1 8500 1 Unknown 1 +F M College 1 9500 1 Good 1 +F M Primary 1 500 1 Unknown 1 +F M Primary 1 1000 1 Good 1 +F M Primary 1 4500 1 Low Risk 1 +F M Primary 1 5000 1 High Risk 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q7.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q7.sql.out new file mode 100644 index 0000000000000..6c4ec28f8ddc3 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q7.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAACAAAA 60.0 52.450000 0.000000 13.630000 +AAAAAAAAAAAEAAAA 96.0 51.560000 0.000000 46.400000 +AAAAAAAAAABEAAAA 8.0 135.430000 0.000000 105.630000 +AAAAAAAAAACAAAAA 43.0 90.780000 0.000000 17.240000 +AAAAAAAAAACCAAAA 31.0 80.140000 0.000000 74.530000 +AAAAAAAAAADBAAAA 79.0 99.440000 2573.530000 90.490000 +AAAAAAAAAADCAAAA 68.0 115.045000 0.000000 72.910000 +AAAAAAAAAAEAAAAA 42.0 16.800000 0.000000 13.770000 +AAAAAAAAAAEDAAAA 42.5 52.175000 0.000000 21.300000 +AAAAAAAAAAFAAAAA 32.0 24.750000 0.000000 23.510000 +AAAAAAAAAAGBAAAA 48.0 119.350000 0.000000 59.670000 +AAAAAAAAAAGCAAAA 54.0 39.420000 964.260000 30.305000 +AAAAAAAAAAHBAAAA 59.333333333333336 86.790000 198.126667 47.576667 +AAAAAAAAAAHDAAAA 82.0 26.030000 981.850000 26.030000 +AAAAAAAAAAKAAAAA 51.666666666666664 47.246667 0.000000 5.333333 +AAAAAAAAAAKBAAAA 11.0 114.560000 0.000000 90.500000 +AAAAAAAAAAKDAAAA 27.0 23.140000 0.000000 20.820000 +AAAAAAAAAALAAAAA 38.0 83.580000 0.000000 51.810000 +AAAAAAAAAALCAAAA 51.0 26.610000 0.000000 19.950000 +AAAAAAAAAAMBAAAA 79.0 41.660000 0.000000 38.740000 +AAAAAAAAAAMCAAAA 70.0 83.210000 0.000000 49.090000 +AAAAAAAAAANAAAAA 2.5 97.780000 0.000000 77.765000 +AAAAAAAAAAOAAAAA 60.0 101.410000 1674.240000 43.600000 +AAAAAAAAAAOCAAAA 7.0 185.510000 0.000000 109.450000 +AAAAAAAAAAPBAAAA 51.5 57.890000 839.850000 52.465000 +AAAAAAAAABABAAAA 20.0 19.300000 0.000000 0.770000 +AAAAAAAAABADAAAA 95.0 100.800000 0.000000 35.280000 +AAAAAAAAABAEAAAA 55.0 2.220000 0.000000 2.080000 +AAAAAAAAABBAAAAA 46.0 61.386667 0.000000 27.306667 +AAAAAAAAABBDAAAA 35.0 166.510000 166.080000 158.180000 +AAAAAAAAABCBAAAA 32.0 33.820000 175.495000 13.025000 +AAAAAAAAABDDAAAA 38.666666666666664 48.240000 1.143333 27.163333 +AAAAAAAAABDEAAAA 8.0 135.240000 38.095000 18.845000 +AAAAAAAAABECAAAA 13.0 73.870000 0.000000 64.260000 +AAAAAAAAABEDAAAA 98.0 76.895000 0.000000 25.375000 +AAAAAAAAABGAAAAA 81.0 46.320000 0.000000 26.580000 +AAAAAAAAABGDAAAA 88.0 81.150000 0.000000 43.820000 +AAAAAAAAABGEAAAA 81.0 18.760000 0.000000 7.310000 +AAAAAAAAABHAAAAA 61.5 101.695000 224.313333 33.520000 +AAAAAAAAABHDAAAA 51.0 128.445000 0.000000 83.725000 +AAAAAAAAABIBAAAA 51.0 20.940000 0.000000 6.490000 +AAAAAAAAABICAAAA 82.5 62.635000 0.860000 27.015000 +AAAAAAAAABJAAAAA 18.333333333333332 76.346667 167.953333 44.930000 +AAAAAAAAABKAAAAA 77.0 122.740000 0.000000 98.190000 +AAAAAAAAABKDAAAA 42.0 125.690000 1504.230000 37.700000 +AAAAAAAAABLBAAAA 59.0 118.240000 72.480000 9.450000 +AAAAAAAAABMBAAAA 97.5 91.465000 0.000000 60.660000 +AAAAAAAAABMDAAAA 46.0 171.290000 0.000000 92.490000 +AAAAAAAAABNCAAAA 20.0 109.530000 0.000000 97.480000 +AAAAAAAAABPAAAAA 59.25 47.130000 62.827500 29.995000 +AAAAAAAAABPBAAAA 66.0 53.950000 30.970000 46.930000 +AAAAAAAAABPDAAAA 66.5 52.420000 0.000000 30.925000 +AAAAAAAAACAAAAAA 45.0 75.860000 0.000000 22.750000 +AAAAAAAAACBBAAAA 64.5 27.585000 0.000000 16.820000 +AAAAAAAAACBCAAAA 71.0 47.330000 0.000000 18.450000 +AAAAAAAAACCDAAAA 68.0 92.350000 0.000000 41.550000 +AAAAAAAAACDAAAAA 21.0 64.390000 59.620000 13.520000 +AAAAAAAAACDDAAAA 65.66666666666667 34.473333 23.316667 14.916667 +AAAAAAAAACEBAAAA 48.0 111.330000 0.000000 12.240000 +AAAAAAAAACECAAAA 67.5 36.365000 373.635000 26.500000 +AAAAAAAAACFBAAAA 9.0 19.600000 0.000000 1.370000 +AAAAAAAAACFDAAAA 26.0 18.350000 0.000000 13.390000 +AAAAAAAAACGCAAAA 60.5 100.270000 0.000000 45.862500 +AAAAAAAAACGDAAAA 58.5 32.805000 396.170000 12.550000 +AAAAAAAAACHBAAAA 97.0 5.380000 27.470000 0.480000 +AAAAAAAAACIAAAAA 57.0 56.880000 0.000000 52.750000 +AAAAAAAAACIBAAAA 58.0 26.060000 152.250000 12.500000 +AAAAAAAAACIDAAAA 71.0 4.020000 0.000000 2.210000 +AAAAAAAAACJAAAAA 30.0 46.910000 0.000000 8.910000 +AAAAAAAAACJDAAAA 63.5 112.305000 0.000000 30.610000 +AAAAAAAAACKCAAAA 50.666666666666664 63.280000 0.000000 26.126667 +AAAAAAAAACLAAAAA 95.0 22.310000 0.000000 0.000000 +AAAAAAAAACLBAAAA 88.0 27.450000 0.000000 23.880000 +AAAAAAAAACMAAAAA 86.0 90.600000 1511.530000 87.880000 +AAAAAAAAACNBAAAA 44.666666666666664 73.076667 52.400000 54.546667 +AAAAAAAAACNCAAAA 62.0 105.380000 0.000000 70.600000 +AAAAAAAAACOAAAAA 41.0 42.810000 412.370000 10.700000 +AAAAAAAAACOBAAAA 36.5 66.735000 14.770000 33.725000 +AAAAAAAAACODAAAA 25.0 65.280000 238.920000 39.820000 +AAAAAAAAACPAAAAA 88.0 164.900000 104.470000 14.840000 +AAAAAAAAACPDAAAA 10.0 78.550000 0.000000 10.990000 +AAAAAAAAADABAAAA 81.0 60.225000 0.000000 43.775000 +AAAAAAAAADBDAAAA 2.0 94.600000 0.000000 59.590000 +AAAAAAAAADCAAAAA 52.0 72.480000 98.720000 15.350000 +AAAAAAAAADCCAAAA 89.0 61.690000 0.000000 59.220000 +AAAAAAAAADCDAAAA 27.5 126.585000 0.000000 61.340000 +AAAAAAAAADDBAAAA 64.0 19.570000 934.160000 16.040000 +AAAAAAAAADDEAAAA 65.0 78.255000 348.165000 49.595000 +AAAAAAAAADEAAAAA 65.0 108.640000 0.000000 91.250000 +AAAAAAAAADEBAAAA 70.0 104.400000 1189.580000 45.930000 +AAAAAAAAADEDAAAA 31.0 130.200000 0.000000 76.810000 +AAAAAAAAADFAAAAA 80.0 5.240000 0.000000 1.310000 +AAAAAAAAADFCAAAA 14.0 9.110000 0.000000 6.650000 +AAAAAAAAADGCAAAA 10.0 84.260000 0.000000 9.260000 +AAAAAAAAADGEAAAA 37.333333333333336 34.403333 101.936667 17.150000 +AAAAAAAAADHBAAAA 32.333333333333336 65.606667 751.803333 54.530000 +AAAAAAAAADJBAAAA 57.0 63.110000 0.000000 47.960000 +AAAAAAAAADKBAAAA 44.0 44.980000 0.000000 3.590000 +AAAAAAAAADLAAAAA 21.0 17.740000 0.000000 9.930000 +AAAAAAAAADMBAAAA 29.0 102.770000 2052.360000 72.960000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q70.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q70.sql.out new file mode 100644 index 0000000000000..a28271a34dcb1 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q70.sql.out @@ -0,0 +1,8 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +-439591881.24 NULL NULL 2 1 +-439591881.24 TN NULL 1 1 +-439591881.24 TN Williamson County 0 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q71.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q71.sql.out new file mode 100644 index 0000000000000..502ee6a80b7ad --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q71.sql.out @@ -0,0 +1,1195 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +10016002 corpamalgamalg #x 19 7 23183.08 +10005002 scholarunivamalg #x 8 8 17109.63 +8014006 edu packmaxi #x 9 53 16420.75 +3003002 exportiexporti #x 17 44 16337.23 +3001001 amalgexporti #x 17 46 16088.53 +4001001 amalgedu pack #x 9 28 15557.83 +6015001 scholarbrand #x 17 50 15411.90 +6007007 brandcorp #x 19 25 15067.50 +6007007 brandcorp #x 18 59 14974.32 +8014006 edu packmaxi #x 19 40 14889.35 +4004002 edu packedu pack #x 18 4 14418.00 +3002002 importoexporti #x 19 40 14063.50 +3003001 exportiexporti #x 19 58 13900.80 +7014010 edu packnameless #x 9 52 13795.60 +2004002 edu packimporto #x 19 17 13663.76 +9013003 exportiunivamalg #x 19 24 13511.26 +5001002 amalgscholar #x 9 41 13318.05 +7014003 edu packnameless #x 19 52 12754.50 +5002002 importoscholar #x 7 16 12584.65 +10005002 scholarunivamalg #x 17 19 12325.56 +4003001 exportiedu pack #x 17 0 12313.71 +1004001 edu packamalg #x 9 6 12311.28 +4001001 amalgedu pack #x 18 21 12299.94 +8004002 edu packnameless #x 9 3 12191.36 +10008006 namelessunivamalg #x 19 47 11971.67 +8001007 amalgnameless #x 19 36 11926.98 +9003003 exportimaxi #x 17 10 11905.95 +1003001 exportiamalg #x 17 48 11845.76 +6009008 maxicorp #x 17 1 11772.54 +6015003 scholarbrand #x 9 40 11755.64 +9003003 exportimaxi #x 19 3 11623.24 +9014011 edu packunivamalg #x 19 12 11407.38 +1001001 amalgamalg #x 17 38 11250.06 +7013007 exportinameless #x 7 27 11169.57 +5001002 amalgscholar #x 18 35 11152.35 +9016003 corpunivamalg #x 17 11 10903.08 +2002002 importoimporto #x 8 38 10861.98 +7009003 maxibrand #x 9 44 10687.08 +3002001 importoexporti #x 17 37 10580.80 +9008003 namelessmaxi #x 9 49 10517.90 +8001007 amalgnameless #x 9 22 10416.24 +7008005 namelessbrand #x 18 31 10196.64 +4004002 edu packedu pack #x 18 29 10166.76 +4003001 exportiedu pack #x 8 41 10118.45 +8012007 importomaxi #x 17 47 10079.52 +5001002 amalgscholar #x 9 11 9922.06 +4003002 exportiedu pack #x 9 8 9888.56 +7004005 edu packbrand #x 17 8 9600.00 +9008003 namelessmaxi #x 18 46 9599.75 +9007003 brandmaxi #x 18 23 9492.33 +7014003 edu packnameless #x 19 39 9431.61 +3004001 edu packexporti #x 18 53 9398.02 +3002001 importoexporti #x 17 51 9370.58 +3002001 importoexporti #x 9 58 9365.40 +1002001 importoamalg #x 8 59 9292.90 +7013002 exportinameless #x 6 23 9269.32 +7002009 importobrand #x 17 31 9114.30 +5003001 exportischolar #x 19 22 9074.52 +10007016 brandunivamalg #x 18 38 9003.20 +10005012 scholarunivamalg #x 18 6 8947.68 +9003003 exportimaxi #x 9 0 8849.28 +8002008 importonameless #x 18 59 8721.28 +8002007 importonameless #x 8 51 8694.00 +3004002 edu packexporti #x 8 18 8480.98 +5002002 importoscholar #x 9 33 8431.24 +3004001 edu packexporti #x 19 5 8427.36 +9003003 exportimaxi #x 8 35 8400.48 +4001001 amalgedu pack #x 17 37 8391.68 +9014011 edu packunivamalg #x 7 33 8367.30 +8002007 importonameless #x 19 52 8361.36 +10014005 edu packamalgamalg #x 9 4 8332.00 +10001003 amalgunivamalg #x 17 56 8299.50 +10011008 amalgamalgamalg #x 18 25 8223.96 +9016005 corpunivamalg #x 17 2 8104.92 +8001007 amalgnameless #x 19 42 8077.98 +10014001 edu packamalgamalg #x 19 52 8077.98 +2002002 importoimporto #x 17 52 8045.25 +8005009 scholarnameless #x 8 17 8033.28 +9013003 exportiunivamalg #x 19 38 7994.24 +5001002 amalgscholar #x 19 18 7980.48 +4003001 exportiedu pack #x 8 57 7973.82 +8006010 corpnameless #x 19 41 7864.08 +10005002 scholarunivamalg #x 6 44 7846.24 +4003001 exportiedu pack #x 19 19 7784.72 +9014011 edu packunivamalg #x 17 48 7677.20 +7014010 edu packnameless #x 9 29 7671.96 +4003001 exportiedu pack #x 19 29 7630.71 +4001001 amalgedu pack #x 9 33 7626.44 +2001001 amalgimporto #x 17 46 7444.64 +9004008 edu packmaxi #x 19 53 7429.73 +9010002 univunivamalg #x 17 25 7428.08 +2001001 amalgimporto #x 19 10 7387.20 +8010001 univmaxi #x 17 38 7327.00 +9003003 exportimaxi #x 17 49 7325.40 +4002001 importoedu pack #x 19 29 7273.80 +5001002 amalgscholar #x 19 31 7253.56 +10001007 amalgunivamalg #x 17 35 7188.16 +3002001 importoexporti #x 19 36 7184.41 +1004001 edu packamalg #x 9 39 7168.20 +8006010 corpnameless #x 6 45 7122.15 +9014008 edu packunivamalg #x 19 35 7048.58 +6015001 scholarbrand #x 19 45 7043.40 +7014001 edu packnameless #x 9 9 6921.20 +3001001 amalgexporti #x 17 58 6919.84 +7016008 corpnameless #x 18 20 6866.37 +4004002 edu packedu pack #x 9 23 6819.60 +7013002 exportinameless #x 8 4 6757.74 +4001001 amalgedu pack #x 18 20 6744.87 +7014010 edu packnameless #x 17 23 6714.90 +2002002 importoimporto #x 9 49 6696.46 +3003002 exportiexporti #x 17 19 6690.42 +4004002 edu packedu pack #x 18 45 6669.03 +8003006 exportinameless #x 9 46 6630.78 +9003003 exportimaxi #x 9 48 6603.20 +9003003 exportimaxi #x 8 28 6580.50 +7012007 importonameless #x 18 6 6548.40 +7015007 scholarnameless #x 17 32 6544.80 +3004001 edu packexporti #x 19 27 6510.42 +9001003 amalgmaxi #x 6 57 6506.50 +2001001 amalgimporto #x 19 8 6500.76 +9003003 exportimaxi #x 18 46 6443.99 +9016003 corpunivamalg #x 8 47 6398.37 +2004002 edu packimporto #x 9 25 6339.69 +9010002 univunivamalg #x 8 21 6302.19 +2004002 edu packimporto #x 19 48 6257.70 +5003001 exportischolar #x 17 44 6225.62 +10016002 corpamalgamalg #x 17 45 6222.80 +9004008 edu packmaxi #x 8 46 6201.78 +7015007 scholarnameless #x 8 18 6162.84 +4002001 importoedu pack #x 18 3 6101.83 +1002001 importoamalg #x 9 1 6058.89 +6008001 namelesscorp #x 17 19 6056.16 +3003002 exportiexporti #x 18 48 6054.30 +7010009 univnameless #x 9 8 6013.24 +9016005 corpunivamalg #x 19 14 5985.00 +7015004 scholarnameless #x 19 49 5981.96 +1004001 edu packamalg #x 19 55 5921.48 +7013007 exportinameless #x 7 0 5905.55 +1003001 exportiamalg #x 8 27 5891.34 +5004001 edu packscholar #x 18 15 5875.10 +8010002 univmaxi #x 17 24 5865.60 +8001007 amalgnameless #x 9 35 5839.47 +3004001 edu packexporti #x 8 45 5819.52 +2001001 amalgimporto #x 19 27 5797.79 +8002007 importonameless #x 19 30 5762.88 +9014008 edu packunivamalg #x 17 53 5754.24 +2003001 exportiimporto #x 8 49 5747.50 +7012007 importonameless #x 17 5 5701.60 +3002001 importoexporti #x 19 17 5699.90 +3003001 exportiexporti #x 8 39 5681.76 +7011010 amalgnameless #x 17 56 5639.04 +6008004 namelesscorp #x 8 42 5623.09 +1003001 exportiamalg #x 18 12 5598.45 +8001009 amalgnameless #x 18 33 5576.00 +7009003 maxibrand #x 17 30 5541.85 +10013006 exportiamalgamalg #x 9 48 5522.14 +10014005 edu packamalgamalg #x 7 45 5490.66 +7013002 exportinameless #x 19 13 5482.24 +9003003 exportimaxi #x 19 15 5480.13 +1004001 edu packamalg #x 18 0 5476.76 +3004001 edu packexporti #x 8 31 5466.72 +4002001 importoedu pack #x 18 42 5460.21 +7010005 univnameless #x 18 48 5435.52 +8008006 namelessnameless #x 8 40 5425.28 +1004001 edu packamalg #x 19 56 5418.16 +1003002 exportiamalg #x 18 52 5397.60 +4004002 edu packedu pack #x 17 21 5377.26 +2002002 importoimporto #x 8 18 5372.40 +1004001 edu packamalg #x 17 16 5341.47 +10015013 scholaramalgamalg #x 6 21 5340.06 +2001001 amalgimporto #x 18 33 5337.60 +6002008 importocorp #x 19 36 5328.99 +3004002 edu packexporti #x 18 33 5301.04 +9003003 exportimaxi #x 9 43 5298.43 +7009003 maxibrand #x 9 26 5193.32 +8010001 univmaxi #x 17 48 5172.16 +8001009 amalgnameless #x 17 1 5166.56 +7015007 scholarnameless #x 17 22 5156.10 +10005012 scholarunivamalg #x 9 16 5114.80 +3003001 exportiexporti #x 18 5 5092.80 +3003001 exportiexporti #x 17 41 5084.64 +9007003 brandmaxi #x 17 23 5045.62 +3002001 importoexporti #x 7 13 5010.32 +3002001 importoexporti #x 17 1 4992.44 +3004001 edu packexporti #x 17 11 4980.35 +4004002 edu packedu pack #x 8 43 4979.04 +10001007 amalgunivamalg #x 19 43 4955.52 +6007007 brandcorp #x 19 20 4942.00 +9014008 edu packunivamalg #x 8 24 4912.02 +5001001 amalgscholar #x 18 54 4879.18 +2001001 amalgimporto #x 18 49 4878.45 +7014010 edu packnameless #x 18 34 4873.60 +8001007 amalgnameless #x 19 41 4830.00 +3002001 importoexporti #x 9 44 4827.15 +4004001 edu packedu pack #x 7 3 4804.34 +4001001 amalgedu pack #x 19 28 4773.53 +4004002 edu packedu pack #x 17 26 4734.60 +6015003 scholarbrand #x 18 34 4665.65 +2004001 edu packimporto #x 17 12 4651.86 +5003001 exportischolar #x 18 4 4631.00 +NULL brandmaxi #x 17 37 4616.78 +4001001 amalgedu pack #x 7 32 4597.00 +4001001 amalgedu pack #x 17 13 4588.56 +6012005 importobrand #x 18 50 4575.60 +3002002 importoexporti #x 19 57 4570.02 +3002001 importoexporti #x 19 39 4528.04 +9016005 corpunivamalg #x 17 16 4526.48 +10008016 namelessunivamalg #x 8 28 4506.81 +2003002 exportiimporto #x 17 42 4484.11 +7015007 scholarnameless #x 18 19 4469.40 +3003002 exportiexporti #x 6 39 4465.80 +10005012 scholarunivamalg #x 18 59 4437.00 +1004001 edu packamalg #x 19 23 4432.40 +2003001 exportiimporto #x 17 19 4421.20 +3004002 edu packexporti #x 18 46 4403.07 +10011008 amalgamalgamalg #x 7 42 4402.30 +4003001 exportiedu pack #x 8 13 4392.08 +8005009 scholarnameless #x 18 0 4379.70 +4003001 exportiedu pack #x 19 50 4349.40 +7002009 importobrand #x 17 33 4341.12 +7006007 corpbrand #x 19 42 4310.33 +1003001 exportiamalg #x 17 21 4301.91 +10008007 namelessunivamalg #x 17 34 4292.47 +8015004 scholarmaxi #x 18 58 4180.80 +2002001 importoimporto #x 18 40 4176.56 +2002002 importoimporto #x 9 54 4173.54 +3001001 amalgexporti #x 9 4 4165.80 +2004001 edu packimporto #x 17 43 4164.04 +3002001 importoexporti #x 17 30 4151.64 +10011008 amalgamalgamalg #x 18 39 4149.63 +4004002 edu packedu pack #x 17 38 4133.66 +8001009 amalgnameless #x 6 43 4122.69 +3002001 importoexporti #x 19 22 4115.04 +3002001 importoexporti #x 19 18 4097.52 +4004002 edu packedu pack #x 19 45 4076.16 +1003001 exportiamalg #x 8 31 4075.59 +7015004 scholarnameless #x 17 5 4068.78 +6009008 maxicorp #x 17 18 4024.77 +7012007 importonameless #x 17 51 4005.99 +7009003 maxibrand #x 18 39 4005.48 +9013005 exportiunivamalg #x 8 42 3946.02 +1004001 edu packamalg #x 9 54 3944.02 +1002002 importoamalg #x 19 45 3925.44 +6005004 scholarcorp #x 9 37 3921.38 +8002007 importonameless #x 18 45 3842.84 +1002001 importoamalg #x 9 30 3842.56 +2002002 importoimporto #x 8 41 3837.18 +8013007 exportimaxi #x 17 41 3815.46 +10005012 scholarunivamalg #x 6 34 3806.53 +6005003 scholarcorp #x 19 28 3796.65 +4004002 edu packedu pack #x 18 28 3780.70 +4004001 edu packedu pack #x 19 7 3777.74 +3003001 exportiexporti #x 9 41 3754.92 +5001002 amalgscholar #x 8 15 3754.40 +1003001 exportiamalg #x 17 53 3748.70 +6007007 brandcorp #x 18 15 3697.20 +7015004 scholarnameless #x 7 52 3666.06 +9014008 edu packunivamalg #x 18 52 3645.18 +4003001 exportiedu pack #x 18 8 3619.35 +3001001 amalgexporti #x 19 54 3606.90 +6009008 maxicorp #x 18 19 3556.14 +5001002 amalgscholar #x 9 4 3522.24 +2004001 edu packimporto #x 17 55 3502.84 +6011001 amalgbrand #x 8 10 3499.45 +3003001 exportiexporti #x 19 55 3482.00 +2003001 exportiimporto #x 17 53 3472.00 +2001001 amalgimporto #x 19 53 3459.52 +2001001 amalgimporto #x 9 49 3457.48 +4001001 amalgedu pack #x 17 39 3433.56 +1004001 edu packamalg #x 19 50 3426.37 +6002008 importocorp #x 18 6 3412.64 +7014010 edu packnameless #x 9 18 3412.08 +7012008 importonameless #x 18 4 3410.74 +3002001 importoexporti #x 17 55 3405.60 +5002001 importoscholar #x 19 17 3400.38 +2001001 amalgimporto #x 17 57 3397.19 +10014001 edu packamalgamalg #x 19 31 3392.50 +7006007 corpbrand #x 17 10 3376.96 +10001003 amalgunivamalg #x 19 41 3352.25 +5001002 amalgscholar #x 19 44 3340.22 +8009005 maxinameless #x 9 24 3338.52 +4002002 importoedu pack #x 19 17 3331.64 +6011003 amalgbrand #x 9 50 3319.68 +4003002 exportiedu pack #x 19 38 3314.22 +1002001 importoamalg #x 19 46 3308.76 +NULL brandmaxi #x 17 53 3303.10 +3004001 edu packexporti #x 19 53 3300.96 +4001001 amalgedu pack #x 17 45 3295.24 +5002001 importoscholar #x 8 11 3271.52 +9012009 importounivamalg #x 17 7 3268.00 +8010001 univmaxi #x 17 31 3267.20 +8010001 univmaxi #x 17 39 3263.40 +9013003 exportiunivamalg #x 17 52 3260.32 +10011008 amalgamalgamalg #x 17 41 3255.54 +6003004 exporticorp #x 19 25 3249.90 +7013002 exportinameless #x 19 39 3219.00 +4003001 exportiedu pack #x 7 6 3211.20 +4001001 amalgedu pack #x 17 49 3193.61 +9009009 maximaxi #x 18 41 3176.58 +4002001 importoedu pack #x 19 11 3175.90 +2001001 amalgimporto #x 17 47 3157.44 +9014011 edu packunivamalg #x 8 58 3137.94 +3002001 importoexporti #x 19 8 3127.98 +7002009 importobrand #x 17 51 3104.96 +10001003 amalgunivamalg #x 17 0 3099.87 +6007007 brandcorp #x 17 2 3096.72 +7014010 edu packnameless #x 19 17 3081.42 +10005012 scholarunivamalg #x 8 39 3081.11 +5001002 amalgscholar #x 9 43 3079.14 +4002001 importoedu pack #x 6 35 3071.00 +4001001 amalgedu pack #x 18 56 3070.71 +7008005 namelessbrand #x 8 5 3070.54 +4003001 exportiedu pack #x 17 14 3060.92 +4004002 edu packedu pack #x 18 30 3026.87 +6015003 scholarbrand #x 19 16 3021.75 +1002001 importoamalg #x 18 23 3018.06 +1003001 exportiamalg #x 17 12 3016.20 +8008006 namelessnameless #x 9 19 3006.85 +8005004 scholarnameless #x 8 7 3004.80 +9008003 namelessmaxi #x 8 13 3004.80 +2004002 edu packimporto #x 17 25 2991.85 +8005009 scholarnameless #x 6 44 2991.08 +7016008 corpnameless #x 8 20 2987.85 +1002001 importoamalg #x 18 38 2983.83 +9016011 corpunivamalg #x 7 43 2923.80 +9016011 corpunivamalg #x 17 56 2910.60 +3003002 exportiexporti #x 17 37 2908.44 +1004001 edu packamalg #x 9 40 2907.30 +7011010 amalgnameless #x 9 39 2899.84 +8012007 importomaxi #x 8 37 2858.82 +4002001 importoedu pack #x 18 7 2846.40 +7010005 univnameless #x 19 20 2838.58 +7002002 importobrand #x 9 28 2829.06 +6011001 amalgbrand #x 19 23 2824.58 +10005012 scholarunivamalg #x 18 51 2800.24 +3001001 amalgexporti #x 18 58 2767.91 +9003003 exportimaxi #x 8 46 2762.53 +4003001 exportiedu pack #x 19 55 2748.48 +7014001 edu packnameless #x 17 15 2742.96 +5002001 importoscholar #x 19 14 2740.79 +7006005 corpbrand #x 17 5 2734.45 +5001002 amalgscholar #x 19 24 2708.16 +7015007 scholarnameless #x 18 6 2707.96 +3004001 edu packexporti #x 19 0 2705.12 +4001001 amalgedu pack #x 18 45 2693.82 +3003001 exportiexporti #x 19 15 2676.38 +7015007 scholarnameless #x 17 51 2675.55 +8010001 univmaxi #x 18 15 2671.00 +3004001 edu packexporti #x 18 37 2668.55 +9015011 scholarunivamalg #x 9 27 2666.22 +8002008 importonameless #x 18 53 2666.00 +6009008 maxicorp #x 19 14 2663.54 +3002001 importoexporti #x 19 47 2647.33 +5002001 importoscholar #x 19 37 2639.14 +3003001 exportiexporti #x 9 1 2634.17 +3004001 edu packexporti #x 18 51 2628.00 +3002001 importoexporti #x 19 30 2612.00 +2001001 amalgimporto #x 18 53 2610.58 +1003001 exportiamalg #x 17 44 2592.59 +3002001 importoexporti #x 9 47 2580.48 +10009010 maxiunivamalg #x 18 53 2579.82 +6008001 namelesscorp #x 17 29 2579.57 +8012007 importomaxi #x 17 52 2569.00 +2002002 importoimporto #x 18 55 2555.57 +8013007 exportimaxi #x 17 23 2545.20 +7006001 corpbrand #x 17 36 2531.43 +10014001 edu packamalgamalg #x 9 26 2527.24 +1003001 exportiamalg #x 17 2 2517.48 +9008003 namelessmaxi #x 8 35 2485.56 +8008006 namelessnameless #x 19 13 2475.20 +10003008 exportiunivamalg #x 8 6 2453.95 +1001001 amalgamalg #x 17 26 2448.16 +7010009 univnameless #x 18 53 2437.00 +9010002 univunivamalg #x 19 27 2435.80 +4001001 amalgedu pack #x 18 38 2431.24 +4004001 edu packedu pack #x 17 33 2426.48 +6012005 importobrand #x 17 41 2420.71 +8013007 exportimaxi #x 19 6 2420.00 +9016005 corpunivamalg #x 18 54 2415.70 +2003001 exportiimporto #x 6 25 2413.20 +2002002 importoimporto #x 18 6 2390.73 +9016008 corpunivamalg #x 17 9 2390.70 +10015013 scholaramalgamalg #x 8 28 2368.85 +4001001 amalgedu pack #x 17 20 2367.30 +8008006 namelessnameless #x 8 20 2363.24 +9012009 importounivamalg #x 8 3 2348.10 +1002002 importoamalg #x 9 44 2336.62 +6007007 brandcorp #x 9 23 2334.24 +1003001 exportiamalg #x 9 54 2325.44 +3004002 edu packexporti #x 9 16 2313.30 +4004001 edu packedu pack #x 19 17 2307.75 +3004001 edu packexporti #x 8 15 2299.77 +7004005 edu packbrand #x 9 34 2294.12 +4003002 exportiedu pack #x 18 55 2280.73 +6005004 scholarcorp #x 9 40 2279.25 +8015004 scholarmaxi #x 9 47 2276.40 +4004002 edu packedu pack #x 19 3 2271.28 +9015011 scholarunivamalg #x 17 54 2269.76 +1002001 importoamalg #x 19 39 2258.08 +6002008 importocorp #x 17 53 2247.20 +9013005 exportiunivamalg #x 7 55 2234.00 +1003002 exportiamalg #x 6 59 2231.46 +10016008 corpamalgamalg #x 17 17 2230.20 +4002001 importoedu pack #x 19 24 2220.00 +4004002 edu packedu pack #x 19 5 2215.50 +9004008 edu packmaxi #x 17 15 2206.40 +4004002 edu packedu pack #x 17 52 2204.16 +2002002 importoimporto #x 17 22 2199.69 +2003002 exportiimporto #x 17 14 2197.84 +8004002 edu packnameless #x 18 33 2188.04 +3003001 exportiexporti #x 17 58 2187.64 +6005003 scholarcorp #x 17 15 2180.85 +8001009 amalgnameless #x 18 41 2178.21 +2002001 importoimporto #x 17 34 2177.28 +6011003 amalgbrand #x 18 48 2176.72 +6008004 namelesscorp #x 17 17 2162.08 +5004002 edu packscholar #x 9 20 2158.40 +6005003 scholarcorp #x 18 7 2152.57 +8010001 univmaxi #x 18 21 2147.42 +9016003 corpunivamalg #x 9 4 2146.05 +9012009 importounivamalg #x 9 6 2138.08 +1003002 exportiamalg #x 9 22 2133.04 +4004002 edu packedu pack #x 18 15 2126.52 +8006010 corpnameless #x 18 41 2126.29 +7013007 exportinameless #x 9 51 2114.10 +10009016 maxiunivamalg #x 9 14 2113.20 +8004002 edu packnameless #x 8 32 2111.54 +3003002 exportiexporti #x 17 45 2106.00 +9008003 namelessmaxi #x 17 55 2100.24 +3004001 edu packexporti #x 8 7 2099.20 +9016011 corpunivamalg #x 19 3 2094.48 +9007003 brandmaxi #x 17 38 2081.01 +7013007 exportinameless #x 8 47 2079.36 +2004001 edu packimporto #x 8 45 2069.88 +2002002 importoimporto #x 8 39 2067.52 +7012007 importonameless #x 7 30 2056.54 +9001003 amalgmaxi #x 19 35 2046.96 +4002001 importoedu pack #x 7 52 2027.30 +9013003 exportiunivamalg #x 17 7 2015.02 +4003002 exportiedu pack #x 9 26 2007.44 +10009016 maxiunivamalg #x 9 57 1997.28 +10008006 namelessunivamalg #x 18 6 1989.49 +5002001 importoscholar #x 19 39 1969.54 +2004001 edu packimporto #x 17 47 1968.80 +9003003 exportimaxi #x 9 22 1964.48 +10009016 maxiunivamalg #x 9 47 1942.08 +2003001 exportiimporto #x 9 48 1919.01 +3003001 exportiexporti #x 18 44 1914.96 +10008016 namelessunivamalg #x 18 1 1902.04 +4003001 exportiedu pack #x 8 27 1899.78 +1003001 exportiamalg #x 7 16 1889.55 +1002001 importoamalg #x 17 40 1889.16 +3001001 amalgexporti #x 8 45 1888.80 +10003008 exportiunivamalg #x 17 42 1858.50 +4003002 exportiedu pack #x 17 55 1851.30 +3002001 importoexporti #x 18 39 1848.96 +8003006 exportinameless #x 8 57 1847.78 +4003001 exportiedu pack #x 18 10 1835.50 +9014011 edu packunivamalg #x 19 9 1816.08 +6012005 importobrand #x 19 32 1815.98 +7010005 univnameless #x 19 23 1815.60 +6011003 amalgbrand #x 8 50 1808.80 +4001002 amalgedu pack #x 17 10 1800.90 +2002002 importoimporto #x 19 0 1798.16 +7010005 univnameless #x 18 31 1789.30 +7002002 importobrand #x 19 33 1786.47 +4004002 edu packedu pack #x 18 14 1786.36 +1002001 importoamalg #x 17 53 1781.00 +8002008 importonameless #x 18 21 1770.21 +10014001 edu packamalgamalg #x 19 7 1760.54 +8003006 exportinameless #x 19 11 1749.55 +9004008 edu packmaxi #x 19 37 1748.25 +4001001 amalgedu pack #x 9 2 1743.00 +3004001 edu packexporti #x 18 21 1736.00 +7006001 corpbrand #x 9 59 1722.93 +7014001 edu packnameless #x 8 14 1720.33 +1002001 importoamalg #x 19 19 1708.92 +1003001 exportiamalg #x 17 55 1703.00 +1003001 exportiamalg #x 8 17 1697.28 +6005003 scholarcorp #x 19 13 1695.72 +2004002 edu packimporto #x 19 26 1693.89 +5003001 exportischolar #x 9 49 1690.08 +7011010 amalgnameless #x 19 13 1688.23 +10014005 edu packamalgamalg #x 17 18 1679.70 +1001001 amalgamalg #x 19 26 1678.27 +2003002 exportiimporto #x 18 10 1677.90 +9007003 brandmaxi #x 18 29 1676.16 +7013002 exportinameless #x 9 21 1674.33 +9004008 edu packmaxi #x 17 45 1667.47 +7012007 importonameless #x 9 9 1662.44 +9010002 univunivamalg #x 8 32 1636.20 +1004001 edu packamalg #x 19 48 1629.36 +6005004 scholarcorp #x 19 28 1623.60 +10014001 edu packamalgamalg #x 19 42 1622.88 +7002002 importobrand #x 19 43 1611.50 +6009008 maxicorp #x 8 40 1607.76 +10008016 namelessunivamalg #x 8 35 1606.32 +2004002 edu packimporto #x 19 41 1600.06 +8013007 exportimaxi #x 19 12 1597.52 +3002001 importoexporti #x 19 5 1597.32 +8013009 exportimaxi #x 8 38 1588.47 +6012005 importobrand #x 18 13 1569.12 +10013006 exportiamalgamalg #x 17 4 1565.76 +2001001 amalgimporto #x 18 22 1560.28 +5002001 importoscholar #x 18 23 1558.80 +1001001 amalgamalg #x 19 13 1542.42 +NULL brandmaxi #x 18 10 1538.24 +10011008 amalgamalgamalg #x 9 49 1536.50 +8014006 edu packmaxi #x 9 49 1534.26 +9013003 exportiunivamalg #x 7 55 1520.45 +7006007 corpbrand #x 8 43 1517.67 +2003001 exportiimporto #x 9 16 1512.00 +10011008 amalgamalgamalg #x 17 46 1506.54 +9016005 corpunivamalg #x 17 54 1505.28 +6012005 importobrand #x 9 47 1503.81 +1002002 importoamalg #x 17 3 1499.84 +9013005 exportiunivamalg #x 18 8 1499.40 +2002002 importoimporto #x 17 14 1483.50 +3001001 amalgexporti #x 17 17 1459.44 +9001003 amalgmaxi #x 19 55 1455.30 +9016011 corpunivamalg #x 18 49 1455.08 +9013003 exportiunivamalg #x 17 2 1443.52 +9003003 exportimaxi #x 9 20 1442.40 +7004005 edu packbrand #x 17 20 1432.67 +8002008 importonameless #x 17 42 1428.68 +9005002 scholarmaxi #x 7 32 1420.70 +10008007 namelessunivamalg #x 9 59 1406.40 +7011010 amalgnameless #x 17 19 1404.86 +7015007 scholarnameless #x 18 46 1402.40 +2001001 amalgimporto #x 17 23 1397.76 +10001007 amalgunivamalg #x 17 42 1388.49 +1003001 exportiamalg #x 17 57 1388.03 +7006001 corpbrand #x 19 58 1387.75 +10005002 scholarunivamalg #x 19 11 1382.68 +9001003 amalgmaxi #x 17 17 1382.16 +3002001 importoexporti #x 6 8 1379.81 +7004005 edu packbrand #x 17 33 1365.52 +3002001 importoexporti #x 18 0 1350.00 +7014003 edu packnameless #x 8 21 1349.76 +9016003 corpunivamalg #x 18 38 1340.64 +2004002 edu packimporto #x 9 4 1334.41 +4003001 exportiedu pack #x 17 55 1326.42 +1002001 importoamalg #x 18 51 1308.82 +6003004 exporticorp #x 17 39 1305.00 +1004001 edu packamalg #x 9 44 1304.91 +10005012 scholarunivamalg #x 6 30 1298.70 +10016008 corpamalgamalg #x 19 32 1293.44 +10008006 namelessunivamalg #x 8 20 1288.16 +3002001 importoexporti #x 19 25 1284.20 +9013005 exportiunivamalg #x 17 50 1269.05 +4003001 exportiedu pack #x 9 12 1266.16 +5001002 amalgscholar #x 17 40 1262.40 +1004001 edu packamalg #x 17 34 1257.90 +10008007 namelessunivamalg #x 8 13 1246.77 +4001001 amalgedu pack #x 9 56 1233.28 +7015007 scholarnameless #x 19 52 1218.28 +9003003 exportimaxi #x 8 39 1211.34 +4003001 exportiedu pack #x 18 3 1208.70 +5004001 edu packscholar #x 18 16 1197.76 +6003004 exporticorp #x 6 50 1195.92 +6002008 importocorp #x 9 14 1194.08 +3003001 exportiexporti #x 17 21 1192.50 +8001009 amalgnameless #x 19 49 1191.50 +6011003 amalgbrand #x 7 32 1189.59 +6009008 maxicorp #x 18 34 1179.80 +2004002 edu packimporto #x 7 19 1178.10 +9013005 exportiunivamalg #x 17 40 1173.15 +7006007 corpbrand #x 8 2 1166.66 +2002002 importoimporto #x 19 17 1162.28 +3001001 amalgexporti #x 18 25 1161.54 +2002001 importoimporto #x 9 59 1161.00 +8010001 univmaxi #x 19 55 1160.64 +4001001 amalgedu pack #x 17 51 1153.88 +9014008 edu packunivamalg #x 18 4 1153.00 +4001001 amalgedu pack #x 19 48 1151.94 +7011010 amalgnameless #x 8 40 1151.64 +5004001 edu packscholar #x 6 57 1148.68 +9004008 edu packmaxi #x 19 3 1140.21 +8001009 amalgnameless #x 17 27 1139.00 +8001007 amalgnameless #x 19 3 1135.20 +9015011 scholarunivamalg #x 9 36 1134.00 +9015011 scholarunivamalg #x 9 41 1133.76 +7010005 univnameless #x 19 8 1130.29 +1001001 amalgamalg #x 18 54 1129.80 +4001001 amalgedu pack #x 8 44 1129.80 +7016008 corpnameless #x 18 27 1128.50 +4002002 importoedu pack #x 17 24 1124.37 +5002002 importoscholar #x 19 20 1124.23 +4002001 importoedu pack #x 9 42 1120.98 +9013005 exportiunivamalg #x 17 16 1113.20 +7015007 scholarnameless #x 17 7 1111.10 +8001009 amalgnameless #x 19 40 1110.42 +2001001 amalgimporto #x 17 19 1108.80 +3004001 edu packexporti #x 18 56 1100.47 +4003001 exportiedu pack #x 9 44 1100.32 +8003006 exportinameless #x 17 54 1098.20 +8001007 amalgnameless #x 19 51 1097.28 +7002002 importobrand #x 17 27 1092.25 +2001001 amalgimporto #x 17 41 1090.98 +2004001 edu packimporto #x 17 14 1088.64 +3003001 exportiexporti #x 17 54 1080.64 +9007003 brandmaxi #x 17 2 1079.52 +1003001 exportiamalg #x 9 36 1079.10 +10008007 namelessunivamalg #x 17 12 1073.16 +6011001 amalgbrand #x 8 53 1071.84 +7008005 namelessbrand #x 19 55 1071.84 +3003001 exportiexporti #x 8 20 1064.85 +7015004 scholarnameless #x 19 59 1059.50 +4004002 edu packedu pack #x 18 19 1052.10 +2001001 amalgimporto #x 9 24 1050.07 +9013005 exportiunivamalg #x 17 19 1049.20 +4001001 amalgedu pack #x 9 3 1047.60 +1004001 edu packamalg #x 17 29 1041.70 +7009003 maxibrand #x 18 17 1040.98 +10008006 namelessunivamalg #x 8 21 1040.04 +9013005 exportiunivamalg #x 9 58 1032.46 +5001002 amalgscholar #x 19 7 1029.76 +9016011 corpunivamalg #x 17 51 1029.34 +1002001 importoamalg #x 18 53 1024.10 +5001001 amalgscholar #x 6 19 1012.69 +2002002 importoimporto #x 17 49 1001.90 +6012005 importobrand #x 17 38 1001.60 +2001001 amalgimporto #x 17 36 992.25 +7008005 namelessbrand #x 19 19 991.36 +5004001 edu packscholar #x 9 9 989.86 +2004001 edu packimporto #x 17 28 984.62 +2003002 exportiimporto #x 19 14 981.54 +3001001 amalgexporti #x 19 24 979.20 +8015004 scholarmaxi #x 9 32 973.50 +9010002 univunivamalg #x 17 40 969.15 +4001001 amalgedu pack #x 9 38 966.92 +3004002 edu packexporti #x 17 39 965.25 +2003001 exportiimporto #x 8 10 960.33 +2002002 importoimporto #x 18 51 958.86 +9016008 corpunivamalg #x 17 52 957.18 +7010005 univnameless #x 17 54 948.42 +9009009 maximaxi #x 8 10 939.60 +4003001 exportiedu pack #x 17 45 937.30 +2004001 edu packimporto #x 17 54 932.94 +9003003 exportimaxi #x 19 36 928.32 +4003002 exportiedu pack #x 9 48 924.80 +8002008 importonameless #x 19 54 915.38 +6002008 importocorp #x 9 0 914.64 +7006007 corpbrand #x 18 2 914.39 +7014010 edu packnameless #x 18 8 910.30 +3004002 edu packexporti #x 18 8 907.12 +5001002 amalgscholar #x 6 8 906.84 +2004001 edu packimporto #x 9 57 906.78 +8013007 exportimaxi #x 17 54 904.68 +7008005 namelessbrand #x 19 29 901.32 +1002001 importoamalg #x 17 45 898.82 +10013006 exportiamalgamalg #x 9 14 896.26 +1003001 exportiamalg #x 18 51 891.87 +9008003 namelessmaxi #x 17 24 890.40 +10008006 namelessunivamalg #x 18 37 889.76 +7014010 edu packnameless #x 8 15 888.12 +8008006 namelessnameless #x 19 3 884.00 +7011010 amalgnameless #x 8 20 883.96 +9013005 exportiunivamalg #x 18 52 883.35 +6015001 scholarbrand #x 9 55 881.45 +3004001 edu packexporti #x 18 0 876.68 +1003001 exportiamalg #x 19 56 874.50 +9014008 edu packunivamalg #x 8 10 873.30 +8003006 exportinameless #x 19 44 863.04 +2001001 amalgimporto #x 8 47 862.92 +3003001 exportiexporti #x 17 48 860.52 +7004005 edu packbrand #x 17 0 857.55 +8004002 edu packnameless #x 6 59 855.84 +8002008 importonameless #x 8 23 855.21 +5001002 amalgscholar #x 7 57 854.70 +6011001 amalgbrand #x 8 7 850.86 +8002007 importonameless #x 17 0 843.50 +2001001 amalgimporto #x 19 21 840.63 +3004001 edu packexporti #x 17 42 836.43 +2004001 edu packimporto #x 17 27 833.91 +4003001 exportiedu pack #x 6 35 825.24 +2001001 amalgimporto #x 9 45 822.00 +2004001 edu packimporto #x 9 15 817.02 +7012007 importonameless #x 19 21 814.45 +6012005 importobrand #x 17 53 805.65 +7011010 amalgnameless #x 18 34 802.80 +4001001 amalgedu pack #x 9 19 799.77 +7014010 edu packnameless #x 9 5 796.86 +4003002 exportiedu pack #x 18 59 794.58 +1001001 amalgamalg #x 17 49 793.50 +2001002 amalgimporto #x 7 46 789.96 +6005003 scholarcorp #x 17 22 788.72 +9013005 exportiunivamalg #x 19 43 788.67 +3004002 edu packexporti #x 19 6 788.12 +3002001 importoexporti #x 18 46 787.76 +9016011 corpunivamalg #x 8 49 787.71 +10014001 edu packamalgamalg #x 19 37 783.36 +1002002 importoamalg #x 18 18 781.60 +5002001 importoscholar #x 17 40 781.55 +5002002 importoscholar #x 18 34 779.22 +7002002 importobrand #x 18 17 773.22 +8013007 exportimaxi #x 7 56 772.80 +8014006 edu packmaxi #x 9 32 771.12 +8008006 namelessnameless #x 9 2 767.36 +5001001 amalgscholar #x 8 51 767.23 +2001001 amalgimporto #x 17 20 765.15 +2001002 amalgimporto #x 18 17 764.40 +9010002 univunivamalg #x 19 2 764.16 +2002002 importoimporto #x 18 46 757.80 +7012008 importonameless #x 17 57 751.74 +9003003 exportimaxi #x 19 24 749.84 +1002002 importoamalg #x 9 12 748.60 +1004001 edu packamalg #x 18 24 746.88 +4001001 amalgedu pack #x 19 46 743.38 +2004001 edu packimporto #x 17 18 742.14 +9013005 exportiunivamalg #x 18 49 739.25 +5004001 edu packscholar #x 17 50 736.65 +7008005 namelessbrand #x 18 7 734.16 +9013005 exportiunivamalg #x 6 58 733.04 +7004005 edu packbrand #x 19 38 728.64 +2004001 edu packimporto #x 19 35 723.36 +5004002 edu packscholar #x 19 57 723.18 +4003001 exportiedu pack #x 8 5 720.16 +8005004 scholarnameless #x 19 1 707.37 +5004002 edu packscholar #x 8 20 704.84 +4001002 amalgedu pack #x 9 46 701.68 +5002001 importoscholar #x 9 42 700.00 +2001001 amalgimporto #x 18 45 698.10 +7002009 importobrand #x 17 55 697.62 +1004001 edu packamalg #x 18 8 694.96 +4003001 exportiedu pack #x 9 11 693.24 +5004001 edu packscholar #x 18 18 693.00 +2004001 edu packimporto #x 9 38 689.92 +6008001 namelesscorp #x 18 21 689.22 +8001007 amalgnameless #x 17 23 685.44 +3002001 importoexporti #x 18 53 677.28 +5003001 exportischolar #x 8 32 675.90 +7009003 maxibrand #x 19 5 672.80 +9013003 exportiunivamalg #x 18 10 672.29 +9014011 edu packunivamalg #x 19 27 671.46 +4004002 edu packedu pack #x 7 46 670.32 +10005002 scholarunivamalg #x 17 42 669.28 +9003003 exportimaxi #x 9 44 666.39 +6005003 scholarcorp #x 9 26 665.28 +8005004 scholarnameless #x 9 14 664.74 +5001002 amalgscholar #x 18 32 657.69 +6011003 amalgbrand #x 9 53 657.57 +7006001 corpbrand #x 8 34 655.68 +6007007 brandcorp #x 17 39 653.90 +8008006 namelessnameless #x 6 9 648.45 +2003001 exportiimporto #x 17 54 645.00 +10008016 namelessunivamalg #x 9 53 638.52 +4003001 exportiedu pack #x 19 4 638.02 +8005004 scholarnameless #x 19 2 633.36 +4001002 amalgedu pack #x 19 11 630.36 +1003001 exportiamalg #x 17 4 629.68 +4004002 edu packedu pack #x 17 15 617.28 +4004002 edu packedu pack #x 18 8 615.36 +9016003 corpunivamalg #x 18 12 609.96 +6007007 brandcorp #x 18 14 607.62 +9016005 corpunivamalg #x 19 54 603.95 +8005009 scholarnameless #x 19 1 601.68 +9009009 maximaxi #x 18 47 598.50 +1002001 importoamalg #x 8 7 593.56 +2003002 exportiimporto #x 17 0 591.43 +3002001 importoexporti #x 17 11 587.94 +4003001 exportiedu pack #x 8 26 587.36 +3002001 importoexporti #x 19 13 586.08 +10013006 exportiamalgamalg #x 18 6 585.12 +9003003 exportimaxi #x 8 4 581.28 +3003001 exportiexporti #x 19 0 577.68 +9007003 brandmaxi #x 17 0 574.52 +3002002 importoexporti #x 9 53 570.78 +10009016 maxiunivamalg #x 19 4 570.57 +1003001 exportiamalg #x 17 59 560.34 +7002002 importobrand #x 8 58 560.28 +7006001 corpbrand #x 7 48 558.90 +NULL brandmaxi #x 18 7 555.45 +6015003 scholarbrand #x 18 2 555.30 +9003003 exportimaxi #x 19 54 546.96 +9003003 exportimaxi #x 9 51 541.80 +1002001 importoamalg #x 19 29 540.00 +6009008 maxicorp #x 17 8 537.37 +1002001 importoamalg #x 17 50 536.58 +7011010 amalgnameless #x 17 34 536.50 +8009005 maxinameless #x 18 44 535.78 +8009005 maxinameless #x 17 9 535.16 +3002001 importoexporti #x 17 44 530.42 +9007003 brandmaxi #x 8 40 529.72 +2001001 amalgimporto #x 9 54 529.55 +2003002 exportiimporto #x 8 41 527.80 +2002002 importoimporto #x 6 9 527.40 +9001003 amalgmaxi #x 17 22 517.16 +2002002 importoimporto #x 19 32 516.40 +3002002 importoexporti #x 9 58 515.78 +7006001 corpbrand #x 8 37 515.08 +4002001 importoedu pack #x 9 0 512.46 +4001001 amalgedu pack #x 18 10 511.88 +9003003 exportimaxi #x 18 59 510.95 +1002002 importoamalg #x 8 10 504.75 +4001001 amalgedu pack #x 9 57 500.32 +2003001 exportiimporto #x 19 55 499.92 +6011003 amalgbrand #x 9 24 498.96 +2001001 amalgimporto #x 19 0 498.18 +2004001 edu packimporto #x 9 35 497.09 +3004002 edu packexporti #x 18 40 495.95 +4001001 amalgedu pack #x 9 34 494.52 +5004002 edu packscholar #x 9 15 491.25 +4002001 importoedu pack #x 8 5 490.56 +10009016 maxiunivamalg #x 17 20 486.96 +7014003 edu packnameless #x 19 23 486.08 +4001001 amalgedu pack #x 9 32 484.72 +5004001 edu packscholar #x 17 24 484.47 +4002001 importoedu pack #x 9 9 477.28 +1004001 edu packamalg #x 8 5 476.13 +6011003 amalgbrand #x 19 9 474.96 +8001009 amalgnameless #x 7 52 473.55 +5001002 amalgscholar #x 17 3 470.64 +3003001 exportiexporti #x 18 10 470.25 +6011001 amalgbrand #x 19 2 467.52 +8009005 maxinameless #x 18 37 467.00 +7015004 scholarnameless #x 17 24 466.86 +1003001 exportiamalg #x 18 7 462.00 +5001002 amalgscholar #x 18 26 461.89 +2001001 amalgimporto #x 18 39 461.70 +9016005 corpunivamalg #x 19 30 458.64 +3002002 importoexporti #x 17 53 454.08 +7014003 edu packnameless #x 8 47 453.84 +2004001 edu packimporto #x 8 27 450.78 +8013007 exportimaxi #x 17 50 449.35 +9013005 exportiunivamalg #x 9 3 448.74 +3004001 edu packexporti #x 8 20 447.76 +3004001 edu packexporti #x 9 57 445.60 +2001002 amalgimporto #x 8 16 444.62 +5003001 exportischolar #x 18 23 443.56 +7010009 univnameless #x 17 24 440.88 +7013007 exportinameless #x 9 47 438.55 +5002001 importoscholar #x 9 45 432.48 +10016008 corpamalgamalg #x 18 41 430.55 +2004002 edu packimporto #x 18 10 430.08 +10009010 maxiunivamalg #x 8 26 429.00 +4003001 exportiedu pack #x 18 42 424.02 +NULL brandmaxi #x 17 16 422.31 +10001007 amalgunivamalg #x 9 36 419.76 +7008005 namelessbrand #x 17 44 419.12 +2004001 edu packimporto #x 8 54 417.34 +4004002 edu packedu pack #x 17 0 415.48 +4003001 exportiedu pack #x 17 9 415.32 +4003002 exportiedu pack #x 18 24 415.16 +4001001 amalgedu pack #x 19 54 414.72 +9014011 edu packunivamalg #x 8 32 413.76 +8001007 amalgnameless #x 19 43 409.08 +3002001 importoexporti #x 17 28 408.90 +4003001 exportiedu pack #x 19 48 406.90 +9013005 exportiunivamalg #x 17 9 400.20 +6003004 exporticorp #x 9 11 399.04 +3003002 exportiexporti #x 17 7 396.00 +10007016 brandunivamalg #x 9 16 394.80 +3002001 importoexporti #x 18 38 393.38 +4001002 amalgedu pack #x 8 57 392.60 +10015013 scholaramalgamalg #x 17 22 390.39 +3002001 importoexporti #x 17 5 390.18 +5004001 edu packscholar #x 9 47 384.09 +3001001 amalgexporti #x 19 19 380.60 +4004002 edu packedu pack #x 18 48 377.60 +7005006 scholarbrand #x 18 52 377.28 +10003008 exportiunivamalg #x 17 8 372.52 +8004002 edu packnameless #x 8 44 372.33 +2003001 exportiimporto #x 18 12 371.25 +3002002 importoexporti #x 19 35 370.59 +1002002 importoamalg #x 18 2 370.08 +6002008 importocorp #x 17 4 369.55 +4003001 exportiedu pack #x 18 51 362.70 +10009010 maxiunivamalg #x 17 40 360.36 +1002002 importoamalg #x 17 35 354.96 +3004001 edu packexporti #x 9 58 354.48 +10003008 exportiunivamalg #x 17 59 352.32 +4001001 amalgedu pack #x 18 48 350.46 +4001001 amalgedu pack #x 17 54 349.44 +4001001 amalgedu pack #x 17 14 347.51 +5001001 amalgscholar #x 19 20 346.40 +4003001 exportiedu pack #x 9 9 345.80 +4001001 amalgedu pack #x 17 21 344.75 +10014005 edu packamalgamalg #x 18 25 344.00 +8008006 namelessnameless #x 17 34 342.72 +9010002 univunivamalg #x 18 53 342.35 +8002007 importonameless #x 8 17 340.80 +1002001 importoamalg #x 7 36 333.32 +4001001 amalgedu pack #x 19 31 330.45 +4004002 edu packedu pack #x 18 44 329.04 +1003001 exportiamalg #x 18 41 326.83 +6008001 namelesscorp #x 19 54 325.65 +7013002 exportinameless #x 17 24 324.24 +4004002 edu packedu pack #x 8 0 323.50 +3002001 importoexporti #x 18 51 323.46 +10016008 corpamalgamalg #x 18 35 322.00 +10014005 edu packamalgamalg #x 17 51 320.76 +1003001 exportiamalg #x 9 56 320.40 +5002002 importoscholar #x 17 54 318.20 +3004001 edu packexporti #x 17 37 317.52 +10007016 brandunivamalg #x 9 6 315.75 +3003001 exportiexporti #x 17 32 314.72 +8010001 univmaxi #x 19 13 314.72 +4001001 amalgedu pack #x 8 0 312.13 +5004002 edu packscholar #x 18 11 310.80 +1002001 importoamalg #x 19 37 309.33 +1002001 importoamalg #x 6 21 308.34 +7006005 corpbrand #x 17 40 307.97 +4001001 amalgedu pack #x 17 18 307.05 +1002001 importoamalg #x 17 24 306.93 +2004001 edu packimporto #x 7 51 305.40 +2004002 edu packimporto #x 17 39 302.90 +9003003 exportimaxi #x 8 20 298.80 +4001001 amalgedu pack #x 19 32 296.38 +5003001 exportischolar #x 19 27 295.95 +2001001 amalgimporto #x 18 4 294.80 +10013006 exportiamalgamalg #x 9 41 290.03 +1002001 importoamalg #x 19 12 281.76 +7013002 exportinameless #x 17 0 279.00 +10005012 scholarunivamalg #x 9 50 275.52 +8010002 univmaxi #x 7 31 272.25 +10015013 scholaramalgamalg #x 7 13 270.28 +1003001 exportiamalg #x 8 47 269.50 +10014005 edu packamalgamalg #x 9 30 267.54 +7011010 amalgnameless #x 19 3 265.47 +9003003 exportimaxi #x 18 55 264.77 +9016003 corpunivamalg #x 9 6 264.64 +2002002 importoimporto #x 18 1 264.30 +1004001 edu packamalg #x 8 13 263.55 +4004001 edu packedu pack #x 6 8 261.08 +8004002 edu packnameless #x 9 56 258.30 +2002002 importoimporto #x 18 36 257.56 +3002001 importoexporti #x 19 20 256.14 +2003002 exportiimporto #x 17 57 255.50 +9010002 univunivamalg #x 19 30 254.52 +6007007 brandcorp #x 9 39 253.23 +9013005 exportiunivamalg #x 7 35 251.93 +3003001 exportiexporti #x 9 42 250.80 +10016002 corpamalgamalg #x 17 47 250.80 +7015007 scholarnameless #x 18 44 249.20 +3004002 edu packexporti #x 18 35 247.28 +8001007 amalgnameless #x 19 32 246.54 +9010002 univunivamalg #x 19 47 246.43 +3002001 importoexporti #x 9 5 244.62 +10016008 corpamalgamalg #x 19 0 243.09 +9009009 maximaxi #x 18 26 242.13 +9008003 namelessmaxi #x 18 9 241.86 +8001009 amalgnameless #x 17 46 241.83 +7013007 exportinameless #x 19 19 241.20 +1002002 importoamalg #x 18 49 241.08 +8005004 scholarnameless #x 8 16 239.98 +4002001 importoedu pack #x 8 45 239.44 +3004001 edu packexporti #x 9 26 237.60 +6012005 importobrand #x 18 8 235.32 +10016008 corpamalgamalg #x 17 27 232.70 +4003002 exportiedu pack #x 8 8 220.08 +1002002 importoamalg #x 18 29 219.36 +2003001 exportiimporto #x 18 31 218.00 +7014010 edu packnameless #x 17 7 217.80 +9005002 scholarmaxi #x 18 41 217.40 +9016003 corpunivamalg #x 17 3 217.00 +NULL brandmaxi #x 18 27 211.40 +2002002 importoimporto #x 17 42 208.98 +4001002 amalgedu pack #x 18 29 206.64 +3002001 importoexporti #x 18 1 205.55 +6009008 maxicorp #x 18 13 204.48 +4001001 amalgedu pack #x 19 35 203.76 +4003002 exportiedu pack #x 9 9 201.41 +1002002 importoamalg #x 6 7 200.80 +9014011 edu packunivamalg #x 17 27 200.72 +8006010 corpnameless #x 9 27 199.75 +3004001 edu packexporti #x 17 18 199.64 +8005009 scholarnameless #x 19 46 198.38 +3004001 edu packexporti #x 9 12 197.62 +6002008 importocorp #x 18 24 196.94 +8013009 exportimaxi #x 17 17 196.50 +10007016 brandunivamalg #x 9 7 188.63 +4002002 importoedu pack #x 17 54 188.25 +4002001 importoedu pack #x 19 55 187.29 +8009005 maxinameless #x 17 0 187.09 +4001001 amalgedu pack #x 17 33 186.10 +1003002 exportiamalg #x 7 56 179.82 +4001001 amalgedu pack #x 9 8 178.40 +3002002 importoexporti #x 17 16 177.30 +8003006 exportinameless #x 9 15 174.05 +9003003 exportimaxi #x 17 52 172.62 +9014011 edu packunivamalg #x 6 12 172.05 +2004001 edu packimporto #x 17 37 170.66 +10001003 amalgunivamalg #x 6 57 169.92 +7005006 scholarbrand #x 7 32 168.08 +6005003 scholarcorp #x 18 59 168.00 +6007007 brandcorp #x 17 22 166.74 +2004001 edu packimporto #x 17 58 165.69 +9003003 exportimaxi #x 8 13 164.78 +6011003 amalgbrand #x 9 43 163.40 +4004002 edu packedu pack #x 7 14 162.92 +2001001 amalgimporto #x 17 0 162.47 +2001002 amalgimporto #x 19 30 161.44 +7013002 exportinameless #x 19 29 160.95 +10011008 amalgamalgamalg #x 17 13 160.56 +9009009 maximaxi #x 17 35 160.40 +3002001 importoexporti #x 17 57 159.60 +2003001 exportiimporto #x 19 37 159.08 +7002009 importobrand #x 19 5 158.40 +7006007 corpbrand #x 8 38 158.16 +4003001 exportiedu pack #x 19 52 157.78 +10009016 maxiunivamalg #x 9 41 157.68 +10015013 scholaramalgamalg #x 17 32 157.62 +6015001 scholarbrand #x 17 0 157.28 +6008004 namelesscorp #x 17 42 156.80 +7013002 exportinameless #x 9 42 156.75 +10014001 edu packamalgamalg #x 18 27 156.18 +2001001 amalgimporto #x 8 5 155.75 +7010005 univnameless #x 9 33 154.64 +7013002 exportinameless #x 8 59 154.38 +5001002 amalgscholar #x 18 31 154.35 +10003008 exportiunivamalg #x 17 27 154.08 +6012005 importobrand #x 9 26 153.51 +10016002 corpamalgamalg #x 17 46 152.10 +5001002 amalgscholar #x 18 41 151.14 +8001009 amalgnameless #x 17 8 150.66 +6008004 namelesscorp #x 19 44 150.40 +6002008 importocorp #x 17 37 150.22 +8004002 edu packnameless #x 19 36 148.23 +8013007 exportimaxi #x 18 33 147.84 +6015003 scholarbrand #x 18 10 147.66 +4002001 importoedu pack #x 19 41 146.60 +4003001 exportiedu pack #x 17 42 146.32 +5003001 exportischolar #x 17 57 142.56 +4004002 edu packedu pack #x 17 4 141.24 +9015011 scholarunivamalg #x 7 56 139.16 +3003001 exportiexporti #x 17 25 135.24 +3003001 exportiexporti #x 9 36 133.92 +8004002 edu packnameless #x 19 42 133.70 +6008004 namelesscorp #x 19 16 132.81 +7014010 edu packnameless #x 17 30 132.46 +2001001 amalgimporto #x 18 6 132.25 +7004005 edu packbrand #x 17 37 132.22 +1003001 exportiamalg #x 19 29 131.58 +8002008 importonameless #x 17 2 131.22 +7012008 importonameless #x 17 53 130.18 +4001001 amalgedu pack #x 7 51 128.50 +8010001 univmaxi #x 17 17 127.28 +9014008 edu packunivamalg #x 9 53 126.70 +8002007 importonameless #x 17 21 126.00 +9004008 edu packmaxi #x 9 19 126.00 +7008005 namelessbrand #x 17 52 125.23 +9003003 exportimaxi #x 17 23 122.74 +6015001 scholarbrand #x 17 36 121.92 +3002002 importoexporti #x 9 40 119.60 +NULL brandmaxi #x 18 20 119.52 +4001001 amalgedu pack #x 19 29 119.38 +4003001 exportiedu pack #x 19 14 119.36 +7015004 scholarnameless #x 17 15 118.36 +2001001 amalgimporto #x 6 35 116.13 +9016005 corpunivamalg #x 9 58 116.09 +4002001 importoedu pack #x 19 19 114.52 +3001001 amalgexporti #x 19 20 114.00 +5001002 amalgscholar #x 19 42 113.04 +3002001 importoexporti #x 6 19 111.52 +3004001 edu packexporti #x 17 45 110.88 +1004001 edu packamalg #x 18 22 110.64 +1002001 importoamalg #x 7 52 110.25 +9003003 exportimaxi #x 18 58 109.34 +6005003 scholarcorp #x 9 27 106.75 +4003002 exportiedu pack #x 9 40 106.44 +8008006 namelessnameless #x 9 39 105.60 +6015003 scholarbrand #x 18 5 104.70 +4002001 importoedu pack #x 18 18 104.00 +7009003 maxibrand #x 18 33 102.00 +10014005 edu packamalgamalg #x 18 19 99.36 +8005004 scholarnameless #x 18 47 98.98 +6008004 namelesscorp #x 17 54 97.40 +2003001 exportiimporto #x 19 0 92.79 +10001007 amalgunivamalg #x 17 1 90.90 +1003001 exportiamalg #x 18 16 90.45 +4003001 exportiedu pack #x 17 4 88.83 +10016002 corpamalgamalg #x 9 14 87.36 +10008016 namelessunivamalg #x 18 7 87.06 +3002002 importoexporti #x 8 24 86.73 +8006010 corpnameless #x 19 44 85.60 +1003001 exportiamalg #x 8 29 84.15 +2001001 amalgimporto #x 18 25 84.04 +9008003 namelessmaxi #x 9 43 83.94 +4003002 exportiedu pack #x 18 2 82.94 +9003003 exportimaxi #x 18 29 81.88 +10003008 exportiunivamalg #x 18 51 81.80 +8006010 corpnameless #x 8 15 80.00 +5004001 edu packscholar #x 8 28 78.75 +10016002 corpamalgamalg #x 19 8 78.75 +8002007 importonameless #x 9 7 78.00 +8001007 amalgnameless #x 9 23 77.04 +3002001 importoexporti #x 6 57 76.38 +10001007 amalgunivamalg #x 8 36 76.20 +8015004 scholarmaxi #x 9 53 74.88 +3003001 exportiexporti #x 9 27 74.52 +9013005 exportiunivamalg #x 8 30 74.20 +9016011 corpunivamalg #x 9 14 72.80 +2002002 importoimporto #x 19 35 70.85 +8005004 scholarnameless #x 9 18 69.60 +9007003 brandmaxi #x 9 18 68.76 +9016008 corpunivamalg #x 9 0 68.64 +9005002 scholarmaxi #x 9 24 67.89 +2001001 amalgimporto #x 18 23 66.65 +5003001 exportischolar #x 17 1 66.36 +7008005 namelessbrand #x 18 3 66.30 +8003006 exportinameless #x 9 8 63.84 +7010005 univnameless #x 19 10 61.88 +4001001 amalgedu pack #x 19 20 60.90 +5002002 importoscholar #x 9 8 58.83 +4001001 amalgedu pack #x 18 27 58.66 +4003002 exportiedu pack #x 9 24 58.56 +3002001 importoexporti #x 9 50 58.32 +8005009 scholarnameless #x 19 10 58.29 +3003001 exportiexporti #x 9 4 57.89 +7014010 edu packnameless #x 9 59 56.76 +7012007 importonameless #x 8 53 55.98 +NULL brandmaxi #x 17 54 55.45 +10007016 brandunivamalg #x 17 11 53.72 +7002002 importobrand #x 7 35 53.52 +7010009 univnameless #x 17 40 53.16 +4002001 importoedu pack #x 8 49 52.87 +4004002 edu packedu pack #x 19 55 50.31 +4001001 amalgedu pack #x 17 46 50.12 +5002001 importoscholar #x 8 10 49.80 +9016005 corpunivamalg #x 19 28 49.50 +4002001 importoedu pack #x 9 22 49.44 +5002002 importoscholar #x 9 38 47.88 +2003001 exportiimporto #x 9 52 47.46 +7016008 corpnameless #x 19 40 47.32 +4004002 edu packedu pack #x 18 34 47.11 +10008006 namelessunivamalg #x 9 37 45.05 +10001003 amalgunivamalg #x 9 14 44.55 +2002002 importoimporto #x 8 9 44.32 +2002002 importoimporto #x 18 10 41.16 +7006007 corpbrand #x 18 6 40.32 +1003001 exportiamalg #x 8 39 38.22 +10001003 amalgunivamalg #x 17 11 37.84 +7012008 importonameless #x 17 1 37.44 +1002001 importoamalg #x 19 15 36.74 +3003001 exportiexporti #x 18 22 34.76 +4001001 amalgedu pack #x 8 32 32.23 +9013005 exportiunivamalg #x 19 33 29.85 +1002001 importoamalg #x 19 57 29.52 +10016002 corpamalgamalg #x 9 18 25.47 +6007007 brandcorp #x 9 11 24.77 +1003002 exportiamalg #x 19 43 24.43 +9004008 edu packmaxi #x 18 57 24.32 +10009016 maxiunivamalg #x 19 19 22.66 +10008006 namelessunivamalg #x 9 3 22.14 +3003001 exportiexporti #x 19 25 21.40 +2001001 amalgimporto #x 8 58 19.88 +3004001 edu packexporti #x 17 29 17.76 +6008001 namelesscorp #x 17 58 17.00 +9009009 maximaxi #x 9 38 16.80 +3003001 exportiexporti #x 17 52 15.96 +6002008 importocorp #x 9 41 15.78 +2004002 edu packimporto #x 8 35 15.21 +6011001 amalgbrand #x 19 40 14.55 +10008016 namelessunivamalg #x 8 54 14.49 +5004001 edu packscholar #x 6 4 13.64 +2003002 exportiimporto #x 8 47 13.04 +2001001 amalgimporto #x 19 59 12.84 +7013002 exportinameless #x 17 1 12.54 +10014001 edu packamalgamalg #x 18 49 12.50 +8015004 scholarmaxi #x 9 0 10.06 +9004008 edu packmaxi #x 9 31 9.80 +5004002 edu packscholar #x 9 46 8.19 +4002002 importoedu pack #x 8 58 7.79 +4001001 amalgedu pack #x 19 59 7.38 +4004002 edu packedu pack #x 9 18 7.33 +2002002 importoimporto #x 19 14 6.54 +5004002 edu packscholar #x 19 19 6.50 +6008001 namelesscorp #x 19 32 5.75 +7010009 univnameless #x 17 29 4.25 +6015003 scholarbrand #x 18 44 3.94 +7014001 edu packnameless #x 18 31 3.08 +6015003 scholarbrand #x 17 18 3.04 +8004002 edu packnameless #x 9 57 0.24 +2001001 amalgimporto #x 17 34 0.00 +2004002 edu packimporto #x 19 40 0.00 +4004002 edu packedu pack #x 17 50 0.00 +6009008 maxicorp #x 17 19 0.00 +6015003 scholarbrand #x 9 51 0.00 +8002007 importonameless #x 19 40 0.00 +9003003 exportimaxi #x 19 33 0.00 +9015011 scholarunivamalg #x 18 10 0.00 +1001001 amalgamalg #x 19 20 NULL +4003001 exportiedu pack #x 9 42 NULL +5002001 importoscholar #x 9 8 NULL +5003001 exportischolar #x 17 43 NULL +8005009 scholarnameless #x 9 58 NULL +8010002 univmaxi #x 9 53 NULL +9013003 exportiunivamalg #x 9 35 NULL +10003008 exportiunivamalg #x 19 30 NULL diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q72.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q72.sql.out new file mode 100644 index 0000000000000..1c0b50b785491 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q72.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL Just good amou 5166 2 2 2 +Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Significantly 5189 2 2 2 +Children shall write only systems. Again outdoor c Operations 5215 2 2 2 +Clouds may compensate about religious man Matches produce 5216 2 2 2 +Critical hours might stress above married, sufficient thousands. Poets shall die medical, ameri Operations 5217 2 2 2 +Customers find for a dogs. Main, following members must live closely because of the interests. Children could imagine more. Innocent, social forces will welcome. I Significantly 5210 2 2 2 +Delicate readers gain too able officers. Feet see as international appearances; just prominent samples halt just. Substantia Just good amou 5197 2 2 2 +Fields might need only however new lengths; explicit, impossible parents cut early able items. Details specify particularly Significantly 5208 2 2 2 +Guests agree around trying, young costs. Here annual banks appeas Just good amou 5204 2 2 2 +Guests agree around trying, young costs. Here annual banks appeas Significantly 5204 2 2 2 +Here extra efforts ensure eyes; merely little periods will not loosen home past a boys. Just local aspects must reclaim. Standard qualities might not roll today. Military, national clothes must go wid Matches produce 5215 2 2 2 +More than familiar lives survive independent reports. Sites must find clearly regulations. Together honest savings refuse so other fingers; british tables Selective, 5215 2 2 2 +Mysterious p Matches produce 5218 2 2 2 +Please separate charges point spiritual, new areas. Angry, able units should try certainly in a accounts. Years retain alternatively. Certain, constant women spend really vital rights. Medical, round Matches produce 5210 2 2 2 +Scottish, forward years could interrupt yesterday pure scienc Just good amou 5214 2 2 2 +Sectors might not know properly. Large, electric workers used to drop even as ca Matches produce 5171 2 2 2 +Sma Matches produce 5204 2 2 2 +Social universities get. Easier yellow results question above basic, direct roots; individual, respective Just good amou 5200 2 2 2 +Upper, usual agencies would not evaluate able, simple faces. Poor lights g Selective, 5197 2 2 2 +NULL Matches produce 5177 1 1 1 +NULL Operations 5203 1 1 1 +NULL Significantly 5171 1 1 1 +NULL Significantly 5203 1 1 1 +NULL Significantly 5213 1 1 1 +A bit liable flowers change also writings. Currently soviet ministers come. Hotels telephone before aggressive, economic eyes. Blue changes improve. Overal Just good amou 5201 1 1 1 +A bit liable flowers change also writings. Currently soviet ministers come. Hotels telephone before aggressive, economic eyes. Blue changes improve. Overal Matches produce 5201 1 1 1 +A lot young materials remain below from a rises. Matches produce 5190 1 1 1 +A lot young materials remain below from a rises. Operations 5190 1 1 1 +A lot young materials remain below from a rises. Selective, 5190 1 1 1 +Aback british songs meet. Chief jobs k Selective, 5178 1 1 1 +Able differences may not spoil particularly then civil ways. Less unusual plants swallow for example in a pp.. Eastern, typical children start to a councils. Exciting cells must meet new, huge me Matches produce 5178 1 1 1 +Able differences may not spoil particularly then civil ways. Less unusual plants swallow for example in a pp.. Eastern, typical children start to a councils. Exciting cells must meet new, huge me Significantly 5167 1 1 1 +Able issues bother however political services. French teachers will act voices. Pregnant, existing cases make by th Just good amou 5208 1 1 1 +Able, available problems apply even to a bodies. Patients so Selective, 5216 1 1 1 +Able, continuous figures see with a patients. Men go more open notes. Different engineers can display. Even strong fortunes cannot put at least general fans; reliable talk Operations 5216 1 1 1 +Able, potential products should Just good amou 5208 1 1 1 +Able, potential products should Operations 5208 1 1 1 +About careful activities hear level cases. However satisfactory reports feel as words. More bad things preserve now poor tories; only strong tools intervene canadian waters. Blin Just good amou 5193 1 1 1 +About right clothes must get thoughtfully to a cases. Eastern improvements Just good amou 5197 1 1 1 +About statistical blocks shall point so brothers. Even new affairs spend hopefully even old contexts. Possible officers wait absolutely with Operations 5168 1 1 1 +Above, new groups will not like much local bodies. However traditional sessions can walk slowly big, young aspects. Quite close companies ought to take in a rules. Leaders must not like of cou Matches produce 5187 1 1 1 +Above, new groups will not like much local bodies. However traditional sessions can walk slowly big, young aspects. Quite close companies ought to take in a rules. Leaders must not like of cou Operations 5187 1 1 1 +Absolutely b Matches produce 5211 1 1 1 +Accidentally wrong communities look more goods. Rural matters recognize. Large, new days go hap Matches produce 5190 1 1 1 +Accidents can include below other, marginal plans. Comparable, welsh exceptions argue most as usual physical claims. Certain months may smell far from in a cases. Active seconds used to restore t Significantly 5213 1 1 1 +Accounts return into a colleagues Operations 5218 1 1 1 +Accurate others could not enable raw goods. Usually early drawings choose originally into a boys. So capable students place Selective, 5188 1 1 1 +Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Just good amou 5209 1 1 1 +Active studies state away already large shelves. Extremely international appli Significantly 5208 1 1 1 +Active, mi Selective, 5192 1 1 1 +Actual things should prevent there responsible schemes. Others go all undoubtedly increasing things. Small, full samples analys Operations 5190 1 1 1 +Added activities leave hands. Nevertheless individual moments could repre Operations 5198 1 1 1 +Added activities leave hands. Nevertheless individual moments could repre Selective, 5174 1 1 1 +Added activities leave hands. Nevertheless individual moments could repre Selective, 5198 1 1 1 +Additional officers shall not apply so poin Just good amou 5217 1 1 1 +Additional, secondary movements will hurt theoretical, major seconds. Families hear well possible, magnetic minutes. Earlier financial women would s Operations 5194 1 1 1 +Addresses retain once more applicable events. Following blocks follow for a develo Operations 5197 1 1 1 +Adequate, true insects clear similar payments. Front rela Just good amou 5177 1 1 1 +Adults might not surrender doubtful, upper industries; earnings insist m Operations 5188 1 1 1 +Afraid questions ta Selective, 5187 1 1 1 +Afraid women shall correct so only women. Red, severe friends repay suddenly out of a elements. Very rigid complaints want however great, little years. Black, itali Selective, 5218 1 1 1 +African, good purposes would determine quite visitors. Sources can make then royal jobs; still full sciences should concentrate no longer elections. Fair applicants talk there under a c Just good amou 5214 1 1 1 +African, good purposes would determine quite visitors. Sources can make then royal jobs; still full sciences should concentrate no longer elections. Fair applicants talk there under a c Selective, 5168 1 1 1 +African, good purposes would determine quite visitors. Sources can make then royal jobs; still full sciences should concentrate no longer elections. Fair applicants talk there under a c Selective, 5214 1 1 1 +Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Significantly 5186 1 1 1 +Again economic objections spend suddenly urgently worried boats. Pupils talk yet nonethele Just good amou 5193 1 1 1 +Again major troubles create new, other children. Fair interactions may telephone Selective, 5199 1 1 1 +Again remote branches should help; processes may s Just good amou 5191 1 1 1 +Again scottish accidents would destroy italian places; please careful services pick sometimes overall men; immensely extra sets move optimistic, substantial actors. Human, likely reports Just good amou 5193 1 1 1 +Again specialist words transform still as perfect forces; expensive, like businesses might want u Just good amou 5191 1 1 1 +Agencies should need likely recommendations. Active, fresh stars shall get just young fragments. Personal Significantly 5216 1 1 1 +Ago natural taxes could protect rats. More local days shall tend closely. Proteins may intervene very perfect men. Procedures make expens Just good amou 5214 1 1 1 +Ago sexual courts may attract. Important, alone observations expect. New, available ways represent years. Excell Operations 5203 1 1 1 +Agricultural, important boys know willingly after the interests. S Significantly 5213 1 1 1 +Agricultural, social tiles would come tragic, various buildings. Good employees shall result high wet plants. Only single contacts support already. Priests would not say unreasonably. Upstairs good Matches produce 5191 1 1 1 +Ahead new columns s Operations 5187 1 1 1 +Ahead young classes should take more central late conservatives. Formal, common details used to think Significantly 5180 1 1 1 +All capital bacteria make jobs. Again appropriate eyes may not leave others. There fixed ways Matches produce 5202 1 1 1 +All direct guns would look cool sure sophisticated bonds; irish sequences look just also local years. Almost close things can look. Build Matches produce 5178 1 1 1 +Allowances might lay at best children. Academic sections burst hot times. Short-term, warm goods Selective, 5177 1 1 1 +Almost busy threats go together recent sides; still tired wines shall not admit on a Matches produce 5209 1 1 1 +Almost good hours should not make. Fully appropriate cases may stop for a terms. Legal compl Matches produce 5188 1 1 1 +Alone bills protect adults. Demands make in a gains. Students train harshly. Ashamed periods choose just general, free places. Senses would finish quite slow, gla Just good amou 5213 1 1 1 +Alone friends would not know else armies. Services recover too extreme, fiscal machines Operations 5205 1 1 1 +Alone new copies discuss to a dates; all black machines get just just royal years. For example free weeks underestimate accurately individual mountains. National, delicious Significantly 5185 1 1 1 +Alone responsibilities used to argue all. Eventual, past reasons lead electrical, absent years. Again big sessions embrace about later familiar hundreds. Certain parts cannot assist desperately good m Just good amou 5208 1 1 1 +Alone responsibilities used to argue all. Eventual, past reasons lead electrical, absent years. Again big sessions embrace about later familiar hundreds. Certain parts cannot assist desperately good m Matches produce 5208 1 1 1 +Alone sole services keep only; stairs shall eliminate for the woods. Methods must need yet. Other students can Just good amou 5203 1 1 1 +Alone, fiscal attitudes will see subsequently. Arrangements used to prefe Selective, 5207 1 1 1 +Alone, fortunate minutes can put particularly out of a consequences. Darling costs run already in a laws. Molecules discover. Temporary, political ty Matches produce 5200 1 1 1 +Already bright poems evaluate somewhere problems; regulations will not conceal now delighted objects; false thoughts follow then. Months should not work only outside times. Fingers prove worker Operations 5170 1 1 1 +Already bright poems evaluate somewhere problems; regulations will not conceal now delighted objects; false thoughts follow then. Months should not work only outside times. Fingers prove worker Significantly 5215 1 1 1 +Also alone patients seem also for the connections. Significant flowers prove finally in a opportunities. Closely international women might avoid tomorrow hidden, following Operations 5209 1 1 1 +Also difficult acres researc Matches produce 5214 1 1 1 +Also dominant elections call only more conventional films. Magazines shall not hand really soon opening hundreds. Equally particula Significantly 5216 1 1 1 +Also eastern matters should not enable now irish, Matches produce 5215 1 1 1 +Also good forms Just good amou 5215 1 1 1 +Also interesting sides acknowledge basically. Tonight low employees run thus. More sympathetic results watch rarely. Even severe arrangements study very Matches produce 5212 1 1 1 +Also invisible shoes sell whit Just good amou 5201 1 1 1 +Also invisible shoes sell whit Selective, 5193 1 1 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q73.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q73.sql.out new file mode 100644 index 0000000000000..615ac241c0e66 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q73.sql.out @@ -0,0 +1,9 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Ransom Thomas Sir N 872 5 +Valle Chandra Dr. N 45338 5 +Richardson Harry Mr. Y 85055 5 +Sauer Larry Mr. N 215795 5 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q74.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q74.sql.out new file mode 100644 index 0000000000000..2e3972217b820 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q74.sql.out @@ -0,0 +1,94 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAECBAAA Frank Wenzel +AAAAAAAAABGKAAAA Jonna King +AAAAAAAAAFAGAAAA Robert Chang +AAAAAAAAAFBNAAAA Robert Baines +AAAAAAAAAGLPAAAA Charlene Marcus +AAAAAAAAAHKEAAAA William Stafford +AAAAAAAABAAGAAAA Luis James +AAAAAAAABBEAAAAA Jason Gallegos +AAAAAAAABGMHBAAA Michael Gillespie +AAAAAAAABIABAAAA Letha Stone +AAAAAAAABIIHAAAA Charles Quarles +AAAAAAAABILCAAAA Theresa Mullins +AAAAAAAABJEDBAAA Arthur Bryan +AAAAAAAACEMIAAAA James Hernandez +AAAAAAAACGLDAAAA Angelo Sloan +AAAAAAAACOEHBAAA Christine Gonzalez +AAAAAAAACPDFBAAA Cheryl Barry +AAAAAAAADFJBBAAA Patrick Jones +AAAAAAAADHCBAAAA Therese Perez +AAAAAAAADHNHBAAA Patrick Cooper +AAAAAAAADKMBAAAA Donald Nelson +AAAAAAAAEBFHAAAA Esther Merrill +AAAAAAAAEFCEBAAA Cornelius Martino +AAAAAAAAEIAHAAAA Henry Desantis +AAAAAAAAEIPIAAAA Luke Rios +AAAAAAAAFAIEAAAA Betty Gipson +AAAAAAAAFDIMAAAA Stephanie Cowan +AAAAAAAAFGMHBAAA Donald Coleman +AAAAAAAAFGNEAAAA Andrew Silva +AAAAAAAAFHNDAAAA Virgil Mcdonald +AAAAAAAAFMOKAAAA Harry Brown +AAAAAAAAFMPPAAAA Manuel Bryant +AAAAAAAAFOEDAAAA Lori Erwin +AAAAAAAAGCGIAAAA Mae Williams +AAAAAAAAGEKLAAAA Jerilyn Walker +AAAAAAAAGGMHAAAA Julia Fisher +AAAAAAAAGLDMAAAA Alex Norris +AAAAAAAAGMFHAAAA Bruce Howe +AAAAAAAAGMGEBAAA Tamika Potts +AAAAAAAAHEIFBAAA NULL Jones +AAAAAAAAHEPFBAAA Kathryn Kinney +AAAAAAAAHGKLAAAA Arthur Christensen +AAAAAAAAHIEIAAAA William Roberts +AAAAAAAAHLEAAAAA Geneva Sims +AAAAAAAAHLJCAAAA Marlene Grover +AAAAAAAAIANDAAAA Elva Wade +AAAAAAAAIBBFBAAA James Compton +AAAAAAAAIBHHBAAA Vennie Loya +AAAAAAAAIBJDBAAA Dean Velez +AAAAAAAAILLJAAAA Billy Ortiz +AAAAAAAAIODCBAAA Jennifer Crane +AAAAAAAAIPGJAAAA Michael NULL +AAAAAAAAIPKJAAAA Charles Jones +AAAAAAAAJADIAAAA Margaret Roberts +AAAAAAAAJBELAAAA Sean Busby +AAAAAAAAJCNBBAAA Johnnie Cox +AAAAAAAAJDEFAAAA Loretta Serrano +AAAAAAAAJGDLAAAA Fredrick Davis +AAAAAAAAJHGFAAAA Pamela Gannon +AAAAAAAAJIAHAAAA Shawna Delgado +AAAAAAAAJINGAAAA Ella Moore +AAAAAAAAJMIDAAAA Sally Thurman +AAAAAAAAKAKPAAAA Carolann Royer +AAAAAAAAKMHPAAAA Robert Jones +AAAAAAAAKNMEBAAA Amber Gonzalez +AAAAAAAALEAHBAAA Eddie Pena +AAAAAAAALMAJAAAA Ileen Linn +AAAAAAAALMGGBAAA Dedra Rainey +AAAAAAAALNLABAAA Janie Garcia +AAAAAAAALPHGBAAA Dorothy Heller +AAAAAAAAMFMKAAAA John Sanders +AAAAAAAAMHOLAAAA Terri Cook +AAAAAAAAMJFAAAAA Marcus Espinal +AAAAAAAAMLOEAAAA Miguel Jackson +AAAAAAAANBECBAAA Michael Lombardi +AAAAAAAANKBBAAAA Diann Saunders +AAAAAAAAOCDCAAAA Armando Jackson +AAAAAAAAOCFLAAAA Bill Freeman +AAAAAAAAOEDIAAAA Alexander Rich +AAAAAAAAOJBPAAAA Jonathan Mcbride +AAAAAAAAOMOKAAAA Laurette Gary +AAAAAAAAOOKKAAAA Deborah Early +AAAAAAAAOPMDAAAA Peggy Smith +AAAAAAAAPAEEBAAA Audria Mattson +AAAAAAAAPBIGBAAA Susie Zavala +AAAAAAAAPEFLAAAA David Martinez +AAAAAAAAPFKDAAAA Linda Simmons +AAAAAAAAPIGBBAAA Charles Welch +AAAAAAAAPNMGAAAA Christine Olds diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q76.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q76.sql.out new file mode 100644 index 0000000000000..c2b51267a1973 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q76.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +catalog NULL 1998 1 Books 8 9808.66 +catalog NULL 1998 1 Children 11 14843.15 +catalog NULL 1998 1 Electronics 12 34659.85 +catalog NULL 1998 1 Home 4 1149.06 +catalog NULL 1998 1 Jewelry 7 16037.86 +catalog NULL 1998 1 Men 17 30340.73 +catalog NULL 1998 1 Music 14 20075.63 +catalog NULL 1998 1 Shoes 10 15432.10 +catalog NULL 1998 1 Sports 4 2660.73 +catalog NULL 1998 1 Women 14 32287.07 +catalog NULL 1998 2 Books 8 27523.09 +catalog NULL 1998 2 Children 11 33106.16 +catalog NULL 1998 2 Electronics 16 24105.87 +catalog NULL 1998 2 Home 14 14282.44 +catalog NULL 1998 2 Jewelry 10 6120.92 +catalog NULL 1998 2 Men 13 25776.26 +catalog NULL 1998 2 Music 11 11113.26 +catalog NULL 1998 2 Shoes 10 5887.74 +catalog NULL 1998 2 Sports 9 15135.88 +catalog NULL 1998 2 Women 7 2373.08 +catalog NULL 1998 3 Books 13 5452.76 +catalog NULL 1998 3 Children 18 32298.89 +catalog NULL 1998 3 Electronics 21 49769.09 +catalog NULL 1998 3 Home 17 14007.10 +catalog NULL 1998 3 Jewelry 23 25680.05 +catalog NULL 1998 3 Men 20 45022.03 +catalog NULL 1998 3 Music 14 20882.65 +catalog NULL 1998 3 Shoes 19 11602.79 +catalog NULL 1998 3 Sports 18 17526.27 +catalog NULL 1998 3 Women 17 20129.34 +catalog NULL 1998 4 NULL 2 607.00 +catalog NULL 1998 4 Books 33 57441.77 +catalog NULL 1998 4 Children 35 53915.78 +catalog NULL 1998 4 Electronics 39 40769.87 +catalog NULL 1998 4 Home 28 40240.88 +catalog NULL 1998 4 Jewelry 38 58972.54 +catalog NULL 1998 4 Men 30 39765.00 +catalog NULL 1998 4 Music 32 70931.52 +catalog NULL 1998 4 Shoes 33 65217.73 +catalog NULL 1998 4 Sports 31 45867.98 +catalog NULL 1998 4 Women 32 33059.29 +catalog NULL 1999 1 Books 11 13083.10 +catalog NULL 1999 1 Children 10 6952.44 +catalog NULL 1999 1 Electronics 10 3112.96 +catalog NULL 1999 1 Home 9 4768.55 +catalog NULL 1999 1 Jewelry 6 7143.30 +catalog NULL 1999 1 Men 10 21187.82 +catalog NULL 1999 1 Music 6 265.20 +catalog NULL 1999 1 Shoes 19 24625.11 +catalog NULL 1999 1 Sports 12 15481.90 +catalog NULL 1999 1 Women 15 25291.30 +catalog NULL 1999 2 Books 17 14284.73 +catalog NULL 1999 2 Children 8 4453.70 +catalog NULL 1999 2 Electronics 14 7808.53 +catalog NULL 1999 2 Home 7 7418.42 +catalog NULL 1999 2 Jewelry 13 11931.85 +catalog NULL 1999 2 Men 8 1730.92 +catalog NULL 1999 2 Music 11 14585.34 +catalog NULL 1999 2 Shoes 10 16267.47 +catalog NULL 1999 2 Sports 7 1058.62 +catalog NULL 1999 2 Women 17 17451.00 +catalog NULL 1999 3 Books 21 34033.74 +catalog NULL 1999 3 Children 15 28755.03 +catalog NULL 1999 3 Electronics 24 41518.93 +catalog NULL 1999 3 Home 20 39919.72 +catalog NULL 1999 3 Jewelry 22 15372.42 +catalog NULL 1999 3 Men 26 49692.31 +catalog NULL 1999 3 Music 23 6840.77 +catalog NULL 1999 3 Shoes 19 21542.78 +catalog NULL 1999 3 Sports 17 15957.19 +catalog NULL 1999 3 Women 27 30416.44 +catalog NULL 1999 4 NULL 1 9077.75 +catalog NULL 1999 4 Books 36 60721.76 +catalog NULL 1999 4 Children 22 21641.02 +catalog NULL 1999 4 Electronics 37 30157.36 +catalog NULL 1999 4 Home 34 42467.56 +catalog NULL 1999 4 Jewelry 35 38566.86 +catalog NULL 1999 4 Men 26 28008.47 +catalog NULL 1999 4 Music 26 31237.65 +catalog NULL 1999 4 Shoes 27 45175.99 +catalog NULL 1999 4 Sports 39 68801.24 +catalog NULL 1999 4 Women 38 55600.11 +catalog NULL 2000 1 Books 12 14558.48 +catalog NULL 2000 1 Children 11 10218.22 +catalog NULL 2000 1 Electronics 6 13621.14 +catalog NULL 2000 1 Home 11 13460.77 +catalog NULL 2000 1 Jewelry 10 8071.78 +catalog NULL 2000 1 Men 10 14579.76 +catalog NULL 2000 1 Music 9 23840.61 +catalog NULL 2000 1 Shoes 15 38230.31 +catalog NULL 2000 1 Sports 13 5259.09 +catalog NULL 2000 1 Women 10 6568.10 +catalog NULL 2000 2 NULL 1 NULL +catalog NULL 2000 2 Books 15 27517.21 +catalog NULL 2000 2 Children 9 4430.92 +catalog NULL 2000 2 Electronics 9 7435.41 +catalog NULL 2000 2 Home 10 12542.28 +catalog NULL 2000 2 Jewelry 8 4325.38 +catalog NULL 2000 2 Men 12 5896.08 +catalog NULL 2000 2 Music 11 2962.75 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q77.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q77.sql.out new file mode 100644 index 0000000000000..aef644cbf749a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q77.sql.out @@ -0,0 +1,49 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 238379361.39 11949589.80 -69066318.65 +catalog channel NULL 116209.49 1989207.49 -1103184.43 +catalog channel NULL 81893158.01 7956829.96 -13266843.17 +catalog channel 1 26819348.55 1989207.49 -4169636.96 +catalog channel 2 27454600.50 1989207.49 -3825432.73 +catalog channel 5 27502999.47 1989207.49 -4168589.05 +store channel NULL 114945147.06 2805428.32 -51038302.63 +store channel 1 19743223.74 437906.57 -8831106.92 +store channel 2 18272722.14 522196.16 -8183951.59 +store channel 4 19720603.73 449683.19 -8686183.94 +store channel 7 19275817.79 456008.74 -8633897.32 +store channel 8 19342554.44 467014.66 -8767463.34 +store channel 10 18590225.22 472619.00 -7935699.52 +web channel NULL 41541056.32 1187331.52 -4761172.85 +web channel 1 1228578.03 47675.10 -188274.21 +web channel 2 1343477.51 44041.96 -110450.27 +web channel 5 1355045.09 11417.70 -99307.08 +web channel 7 1439105.65 44505.99 -131740.09 +web channel 8 1222672.59 64741.73 -137947.41 +web channel 11 1380233.43 12631.13 -89907.09 +web channel 13 1380503.10 66341.09 -190038.15 +web channel 14 1322143.80 66841.97 -172945.60 +web channel 17 1640923.76 6485.63 -15639.73 +web channel 19 1258408.09 43554.94 -95271.34 +web channel 20 1413076.47 25716.55 -176644.94 +web channel 23 1522205.18 26405.14 -102673.55 +web channel 25 1435872.54 58108.85 -141661.85 +web channel 26 1364954.66 41289.03 -226994.92 +web channel 29 1456398.83 23148.07 -127320.23 +web channel 31 1331158.19 58914.20 -259972.34 +web channel 32 1430016.29 68634.13 -216355.06 +web channel 35 1275017.69 9253.48 -86568.79 +web channel 37 1527487.12 34512.52 -145699.04 +web channel 38 1459100.39 77580.83 -145709.91 +web channel 41 1542741.42 22802.83 -126554.81 +web channel 43 1556206.93 43143.17 -170251.07 +web channel 44 1197577.68 47675.73 -232746.02 +web channel 47 1311689.48 14426.63 -151021.48 +web channel 49 1487129.22 38844.09 -129263.73 +web channel 50 1404935.78 52795.27 -203592.12 +web channel 53 1308281.20 25132.82 -234014.24 +web channel 55 1348261.69 34116.67 -208865.51 +web channel 56 1223987.10 55188.28 -252201.97 +web channel 59 1373867.41 21405.99 -191540.30 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q78.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q78.sql.out new file mode 100644 index 0000000000000..df28fcb255ae8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q78.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q79.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q79.sql.out new file mode 100644 index 0000000000000..0a8f10fa2ce4a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q79.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL Fairview 198070 3414.76 -28275.11 +NULL NULL Fairview 178356 4636.16 -24754.68 +NULL NULL Fairview 4561 38.97 -17858.20 +NULL NULL Fairview 227170 4972.68 -17709.46 +NULL NULL Fairview 78433 689.97 -16192.40 +NULL NULL Fairview 150646 3348.61 -15819.67 +NULL NULL Fairview 15261 2836.87 -15688.87 +NULL NULL Fairview 226126 651.62 -14052.62 +NULL NULL Fairview 36397 2615.22 -13738.93 +NULL NULL Fairview 40 532.59 -12157.64 +NULL NULL Fairview 237656 9409.20 -12036.60 +NULL NULL Fairview 178664 0.00 -11859.96 +NULL NULL Fairview 60434 7214.21 -11468.85 +NULL NULL Fairview 23163 1270.90 -11127.55 +NULL NULL Fairview 36041 7796.51 -11067.35 +NULL NULL Fairview 35679 5305.03 -10515.13 +NULL NULL Fairview 233081 361.69 -9831.11 +NULL NULL Fairview 65032 3442.33 -9620.20 +NULL NULL Fairview 162505 7179.34 -8572.34 +NULL NULL Fairview 142857 8377.25 -8392.77 +NULL NULL Fairview 193575 1828.27 -6805.77 +NULL NULL Fairview 49934 228.47 -5430.28 +NULL NULL Fairview 67999 621.15 -5031.30 +NULL NULL Fairview 223653 3570.27 -4327.27 +NULL NULL Fairview 6384 0.00 -3987.67 +NULL NULL Fairview 168833 1894.07 -3677.23 +NULL NULL Fairview 93344 0.00 -1599.51 +NULL NULL Fairview 53781 16.91 764.24 +NULL NULL Fairview 175529 1252.60 1017.06 +NULL NULL Fairview 197495 373.09 2365.53 +NULL NULL Fairview 4047 3121.19 2715.67 +NULL NULL Midway 48072 11113.87 -29510.66 +NULL NULL Midway 11394 5172.08 -25597.50 +NULL NULL Midway 207040 2008.73 -25394.40 +NULL NULL Midway 222765 7629.39 -25247.34 +NULL NULL Midway 217712 4349.03 -24697.82 +NULL NULL Midway 45131 5788.09 -24158.43 +NULL NULL Midway 164632 1394.75 -23255.21 +NULL NULL Midway 127656 5021.65 -22989.97 +NULL NULL Midway 83311 7875.67 -22914.32 +NULL NULL Midway 56977 3159.77 -20447.95 +NULL NULL Midway 215293 0.00 -19925.34 +NULL NULL Midway 140002 905.08 -18610.09 +NULL NULL Midway 173634 9179.91 -17739.01 +NULL NULL Midway 23863 152.72 -17463.88 +NULL NULL Midway 160686 5769.52 -17219.86 +NULL NULL Midway 82279 1835.82 -17174.21 +NULL NULL Midway 112924 10990.60 -16901.50 +NULL NULL Midway 166267 653.18 -16156.26 +NULL NULL Midway 213647 1924.30 -16123.38 +NULL NULL Midway 204741 2259.86 -15336.56 +NULL NULL Midway 122738 12543.65 -15281.12 +NULL NULL Midway 173203 4314.45 -14126.67 +NULL NULL Midway 147660 3460.04 -14088.14 +NULL NULL Midway 204175 1442.30 -13954.82 +NULL NULL Midway 124093 3122.91 -13873.66 +NULL NULL Midway 230300 1956.43 -13769.94 +NULL NULL Midway 217344 0.00 -13534.82 +NULL NULL Midway 91656 2427.25 -13507.46 +NULL NULL Midway 148611 5844.43 -13212.15 +NULL NULL Midway 234650 2836.85 -12426.80 +NULL NULL Midway 170455 3175.00 -11962.07 +NULL NULL Midway 26663 264.59 -11479.46 +NULL NULL Midway 75387 5918.43 -11406.83 +NULL NULL Midway 77944 788.27 -11329.12 +NULL NULL Midway 135318 805.73 -11284.26 +NULL NULL Midway 2896 12623.23 -11061.86 +NULL NULL Midway 158461 29.32 -10931.38 +NULL NULL Midway 1333 1390.53 -10865.55 +NULL NULL Midway 116866 3621.69 -10822.24 +NULL NULL Midway 194993 3401.82 -10654.47 +NULL NULL Midway 187290 6137.11 -10592.41 +NULL NULL Midway 151959 8548.17 -10204.65 +NULL NULL Midway 149775 15.18 -10113.87 +NULL NULL Midway 157401 1094.24 -10070.65 +NULL NULL Midway 210606 2098.02 -9961.51 +NULL NULL Midway 118130 182.41 -9901.81 +NULL NULL Midway 7439 2125.08 -9595.87 +NULL NULL Midway 19796 1532.38 -9553.82 +NULL NULL Midway 222335 0.00 -9523.98 +NULL NULL Midway 208688 1211.17 -9220.71 +NULL NULL Midway 32396 2191.49 -9201.97 +NULL NULL Midway 111809 4648.48 -9170.68 +NULL NULL Midway 12549 1306.17 -8842.14 +NULL NULL Midway 6762 2278.78 -8696.20 +NULL NULL Midway 12892 2172.51 -8615.80 +NULL NULL Midway 133874 0.00 -8601.78 +NULL NULL Midway 86109 161.95 -8348.77 +NULL NULL Midway 58015 5671.25 -8231.58 +NULL NULL Midway 173045 0.00 -8004.30 +NULL NULL Midway 221481 315.96 -7893.04 +NULL NULL Midway 4682 0.00 -7769.63 +NULL NULL Midway 155285 0.00 -7757.04 +NULL NULL Midway 194931 1315.23 -7705.13 +NULL NULL Midway 208424 878.43 -7702.00 +NULL NULL Midway 199504 2052.21 -7662.62 +NULL NULL Midway 73227 2869.94 -7484.05 +NULL NULL Midway 12578 3486.23 -7447.58 +NULL NULL Midway 160835 3133.37 -7258.48 +NULL NULL Midway 164697 6050.09 -7208.84 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q8.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q8.sql.out new file mode 100644 index 0000000000000..17b390542941c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q8.sql.out @@ -0,0 +1,10 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +able -9960913.46 +bar -10200043.73 +eing -11104757.88 +ese -11009853.93 +ought -10574796.20 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q80.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q80.sql.out new file mode 100644 index 0000000000000..bdadd9e04e13c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q80.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 13553636.71 630819.07 -3876883.14 +catalog channel NULL 4818809.23 250952.51 -519182.04 +catalog channel catalog_pageAAAAAAAAAAABAAAA 18407.14 67.62 4548.90 +catalog channel catalog_pageAAAAAAAAABABAAAA 1837.52 0.00 -3388.65 +catalog channel catalog_pageAAAAAAAAACABAAAA 18390.60 0.00 9578.02 +catalog channel catalog_pageAAAAAAAAADABAAAA 1404.55 27.58 -3703.26 +catalog channel catalog_pageAAAAAAAAADCBAAAA 2868.85 0.00 826.15 +catalog channel catalog_pageAAAAAAAAAEABAAAA 5039.78 0.00 -3701.62 +catalog channel catalog_pageAAAAAAAAAECBAAAA 3971.05 0.00 1757.41 +catalog channel catalog_pageAAAAAAAAAFABAAAA 2316.00 0.00 -1002.65 +catalog channel catalog_pageAAAAAAAAAFCBAAAA 7015.40 0.00 915.49 +catalog channel catalog_pageAAAAAAAAAGABAAAA 23067.38 0.00 10478.19 +catalog channel catalog_pageAAAAAAAAAGCBAAAA 27364.22 0.00 8607.28 +catalog channel catalog_pageAAAAAAAAAHABAAAA 12794.77 0.00 -7812.73 +catalog channel catalog_pageAAAAAAAAAHCBAAAA 2640.87 63.18 -2334.42 +catalog channel catalog_pageAAAAAAAAAICBAAAA 1845.15 0.00 -4629.81 +catalog channel catalog_pageAAAAAAAAAJCBAAAA 1555.72 0.00 -5039.18 +catalog channel catalog_pageAAAAAAAAAKPAAAAA 42070.02 0.00 6567.19 +catalog channel catalog_pageAAAAAAAAALCBAAAA 8772.47 0.00 479.78 +catalog channel catalog_pageAAAAAAAAALPAAAAA 20380.13 0.00 2437.83 +catalog channel catalog_pageAAAAAAAAAMCBAAAA 15549.59 5438.72 -7150.88 +catalog channel catalog_pageAAAAAAAAAMPAAAAA 11607.07 0.00 -7275.33 +catalog channel catalog_pageAAAAAAAAANCBAAAA 21588.37 0.00 4221.36 +catalog channel catalog_pageAAAAAAAAANPAAAAA 21638.94 0.00 4905.57 +catalog channel catalog_pageAAAAAAAAAOCBAAAA 521.86 0.00 160.52 +catalog channel catalog_pageAAAAAAAAAOPAAAAA 38265.71 0.00 -15751.13 +catalog channel catalog_pageAAAAAAAAAPCBAAAA 4872.72 0.00 2322.60 +catalog channel catalog_pageAAAAAAAAAPPAAAAA 16581.43 5140.00 -3778.70 +catalog channel catalog_pageAAAAAAAABAABAAAA 6642.83 0.00 768.98 +catalog channel catalog_pageAAAAAAAABBABAAAA 280.96 0.00 -50.56 +catalog channel catalog_pageAAAAAAAABCABAAAA 1048.56 0.00 -600.26 +catalog channel catalog_pageAAAAAAAABDABAAAA 17545.67 0.00 -9838.60 +catalog channel catalog_pageAAAAAAAABDCBAAAA 9090.78 0.00 -572.50 +catalog channel catalog_pageAAAAAAAABEABAAAA 17525.88 232.32 -2410.45 +catalog channel catalog_pageAAAAAAAABECBAAAA 18693.14 0.00 -3538.78 +catalog channel catalog_pageAAAAAAAABFABAAAA 17175.44 61.36 2821.53 +catalog channel catalog_pageAAAAAAAABFCBAAAA 140.82 0.00 -569.12 +catalog channel catalog_pageAAAAAAAABGABAAAA 384.55 0.00 -619.85 +catalog channel catalog_pageAAAAAAAABGCBAAAA 3166.40 1590.00 -9892.69 +catalog channel catalog_pageAAAAAAAABHABAAAA 14266.65 0.00 -4959.21 +catalog channel catalog_pageAAAAAAAABHCBAAAA 1741.48 0.00 403.78 +catalog channel catalog_pageAAAAAAAABICBAAAA 1413.03 0.00 -789.20 +catalog channel catalog_pageAAAAAAAABJCBAAAA 17473.29 18.29 2514.85 +catalog channel catalog_pageAAAAAAAABKCBAAAA 415.15 0.00 -1.90 +catalog channel catalog_pageAAAAAAAABKPAAAAA 2724.27 0.00 -8394.13 +catalog channel catalog_pageAAAAAAAABLCBAAAA 15422.53 0.00 7523.39 +catalog channel catalog_pageAAAAAAAABLPAAAAA 20440.65 0.00 -6095.43 +catalog channel catalog_pageAAAAAAAABMCBAAAA 2324.92 0.00 1256.32 +catalog channel catalog_pageAAAAAAAABMPAAAAA 2322.14 0.00 -4465.41 +catalog channel catalog_pageAAAAAAAABNCBAAAA 10606.89 0.00 -4039.08 +catalog channel catalog_pageAAAAAAAABNPAAAAA 5671.18 0.00 -6038.20 +catalog channel catalog_pageAAAAAAAABOCBAAAA 7494.52 0.00 2578.36 +catalog channel catalog_pageAAAAAAAABOPAAAAA 19163.74 0.00 2677.78 +catalog channel catalog_pageAAAAAAAABPCBAAAA 48.42 0.00 -92.80 +catalog channel catalog_pageAAAAAAAABPPAAAAA 65484.56 8188.50 16714.73 +catalog channel catalog_pageAAAAAAAACAABAAAA 34936.59 0.00 6574.55 +catalog channel catalog_pageAAAAAAAACBABAAAA 8178.17 0.00 1370.92 +catalog channel catalog_pageAAAAAAAACCABAAAA 24428.99 0.00 7616.93 +catalog channel catalog_pageAAAAAAAACDABAAAA 17628.32 1503.67 -2086.58 +catalog channel catalog_pageAAAAAAAACDCBAAAA 222.09 0.00 108.57 +catalog channel catalog_pageAAAAAAAACEABAAAA 9898.50 491.66 -4581.91 +catalog channel catalog_pageAAAAAAAACECBAAAA 11493.96 0.00 -7027.70 +catalog channel catalog_pageAAAAAAAACFABAAAA 25027.83 11283.47 4064.93 +catalog channel catalog_pageAAAAAAAACFCBAAAA 618.24 0.00 -1316.28 +catalog channel catalog_pageAAAAAAAACGABAAAA 1830.05 0.00 -3073.99 +catalog channel catalog_pageAAAAAAAACHABAAAA 1121.15 275.52 -5850.52 +catalog channel catalog_pageAAAAAAAACHCBAAAA 13108.45 0.00 -786.42 +catalog channel catalog_pageAAAAAAAACICBAAAA 1755.49 0.00 146.98 +catalog channel catalog_pageAAAAAAAACJCBAAAA 2982.05 0.00 -196.21 +catalog channel catalog_pageAAAAAAAACKCBAAAA 13658.88 0.00 5924.52 +catalog channel catalog_pageAAAAAAAACKPAAAAA 26319.77 162.81 -8423.49 +catalog channel catalog_pageAAAAAAAACLCBAAAA 6469.97 0.00 -5067.48 +catalog channel catalog_pageAAAAAAAACLPAAAAA 43938.15 7021.98 -4638.29 +catalog channel catalog_pageAAAAAAAACMCBAAAA 9332.13 0.00 -5421.54 +catalog channel catalog_pageAAAAAAAACMPAAAAA 13306.48 0.00 -6449.30 +catalog channel catalog_pageAAAAAAAACNPAAAAA 18488.66 0.00 186.37 +catalog channel catalog_pageAAAAAAAACOCBAAAA 1220.46 451.26 -6770.17 +catalog channel catalog_pageAAAAAAAACOPAAAAA 27469.76 1091.48 7443.78 +catalog channel catalog_pageAAAAAAAACPCBAAAA 117.48 0.00 30.19 +catalog channel catalog_pageAAAAAAAACPPAAAAA 19041.61 172.22 380.68 +catalog channel catalog_pageAAAAAAAADAABAAAA 12391.42 953.40 -7032.35 +catalog channel catalog_pageAAAAAAAADBABAAAA 1495.00 0.00 -9772.65 +catalog channel catalog_pageAAAAAAAADCABAAAA 0.00 0.00 -3388.10 +catalog channel catalog_pageAAAAAAAADDABAAAA 15917.59 0.00 4459.43 +catalog channel catalog_pageAAAAAAAADDCBAAAA 4409.76 733.12 912.46 +catalog channel catalog_pageAAAAAAAADEABAAAA 14545.59 10419.31 -2030.01 +catalog channel catalog_pageAAAAAAAADFABAAAA 5222.68 0.00 -3306.65 +catalog channel catalog_pageAAAAAAAADFCBAAAA 2442.96 0.00 -758.68 +catalog channel catalog_pageAAAAAAAADGABAAAA 14604.59 0.00 -3980.39 +catalog channel catalog_pageAAAAAAAADHABAAAA 19078.94 12277.75 6448.20 +catalog channel catalog_pageAAAAAAAADHCBAAAA 12207.03 0.00 1895.71 +catalog channel catalog_pageAAAAAAAADICBAAAA 12548.28 0.00 407.59 +catalog channel catalog_pageAAAAAAAADJCBAAAA 3526.74 0.00 2014.47 +catalog channel catalog_pageAAAAAAAADKCBAAAA 3071.53 0.00 -5979.53 +catalog channel catalog_pageAAAAAAAADKPAAAAA 52608.41 1959.75 20557.42 +catalog channel catalog_pageAAAAAAAADLCBAAAA 386.12 0.00 -1094.08 +catalog channel catalog_pageAAAAAAAADLPAAAAA 22380.47 230.88 611.66 +catalog channel catalog_pageAAAAAAAADMCBAAAA 4943.75 0.00 2158.10 +catalog channel catalog_pageAAAAAAAADMPAAAAA 17113.94 0.00 4231.57 +catalog channel catalog_pageAAAAAAAADNCBAAAA 982.53 0.00 -4098.99 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q81.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q81.sql.out new file mode 100644 index 0000000000000..0a7e038efaabf --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q81.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAGOAAAA Mrs. Melissa Barton 761 Lincoln Drive Suite 440 Marion Cobb County GA 30399 United States -5.00 condo 2053.35 +AAAAAAAAACCMAAAA Mr. Craig Thompson 607 Lakeview Laurel Lane Suite 140 Hillcrest Floyd County GA 33003 United States -5.00 apartment 2314.62 +AAAAAAAAADBBAAAA Dr. Robert Willis 48 Ninth West Lane Suite 370 Oak Grove Cook County GA 38370 United States -5.00 single family 2232.39 +AAAAAAAAAELEAAAA Miss NULL Werner 800 11th Avenue Suite M Harmony Echols County GA 35804 United States -5.00 condo 2312.04 +AAAAAAAAAFIHBAAA Sir Chadwick Stevens 679 Forest Central Avenue Suite 60 Sulphur Springs Gordon County GA 38354 United States -5.00 condo 2282.39 +AAAAAAAAAHDGAAAA Mr. Justin Garcia 281 Spring Way Suite 320 Shiloh Fannin County GA 39275 United States -5.00 single family 4185.32 +AAAAAAAAAICMAAAA Sir Willie Maldonado 492 Park Street Suite C Shady Grove Chatham County GA 32812 United States -5.00 apartment 2428.92 +AAAAAAAAALPABAAA Dr. John Wang 724 Oak Ave Suite 70 Maple Grove Newton County GA 38252 United States -5.00 single family 1727.64 +AAAAAAAAAMJAAAAA Sir Roy Mark 92 Cedar Sycamore Way Suite 30 Union Hill Jenkins County GA 37746 United States -5.00 single family 1825.38 +AAAAAAAAAMKFBAAA Dr. Lamar Jones 194 Maple Cir. Suite 80 Oakwood Jefferson County GA 30169 United States -5.00 single family 3977.04 +AAAAAAAAAMLOAAAA Ms. Brooke Garcia 914 Hillcrest Cedar Drive Suite 210 Stringtown Jasper County GA 30162 United States -5.00 condo 4133.35 +AAAAAAAAAMODAAAA Mrs. Lori Campos 728 Pine Walnut Cir. Suite M Woodlawn Worth County GA 34098 United States -5.00 condo 4182.98 +AAAAAAAAANJOAAAA Ms. Allen Rodriguez 156 Ash Seventh Way Suite 100 Pleasant Valley Sumter County GA 32477 United States -5.00 apartment 2710.46 +AAAAAAAAAODABAAA Dr. Oscar Miller 651 Smith West Dr. Suite E Antioch Evans County GA 38605 United States -5.00 condo 7234.29 +AAAAAAAAAODIAAAA Mr. Julio Gamble 211 View Cir. Suite Y Summit Bulloch County GA 30499 United States -5.00 single family 3291.69 +AAAAAAAABBAGAAAA Dr. Charles Butts NULL Park Hickory RD NULL NULL NULL GA 38605 NULL -5.00 NULL 3291.71 +AAAAAAAABBCJAAAA Sir Doyle Shaffer 401 Johnson Main Boulevard Suite 150 Shiloh Worth County GA 39275 United States -5.00 single family 4027.79 +AAAAAAAABBJKAAAA Dr. Mildred Hogan 174 Birch Pkwy Suite Q Enterprise Webster County GA 31757 United States -5.00 condo 4006.97 +AAAAAAAABCCEAAAA Mrs. Linda Worth 283 2nd Pkwy Suite 230 Fairfield Chattooga County GA 36192 United States -5.00 condo 5047.24 +AAAAAAAABCEFBAAA Dr. William Parks 19 Meadow Circle Suite P Providence Gwinnett County GA 36614 United States -5.00 apartment 4262.30 +AAAAAAAABCFKAAAA NULL Shawn Peterson 220 Johnson First Ct. Suite F Kingston Randolph County GA 34975 United States -5.00 condo 2786.80 +AAAAAAAABCGEAAAA Miss Margaret Wright 314 5th Washington Street Suite S Forest Hills Dougherty County GA 39237 United States -5.00 single family 2049.52 +AAAAAAAABDKOAAAA Dr. Vernita Bennett 763 13th Court Suite 60 Maple Hill Rabun County GA 38095 United States -5.00 single family 18570.61 +AAAAAAAABEIKAAAA Miss Nena August 297 Lee Lane Suite 240 Cedar Henry County GA 31229 United States -5.00 condo 2094.88 +AAAAAAAABEJLAAAA Mr. Abel Lucero 382 Ash North Parkway Suite J Plainview Catoosa County GA 33683 United States -5.00 single family 3530.52 +AAAAAAAABIDEAAAA Dr. Marcus Holder 724 Oak Ave Suite 70 Maple Grove Newton County GA 38252 United States -5.00 single family 1927.32 +AAAAAAAABIGIAAAA Mrs. Caprice Espinoza 527 Church Second Court Suite M Derby Taylor County GA 37702 United States -5.00 condo 6808.21 +AAAAAAAABJEEAAAA Dr. Clarence Swanson NULL NULL Ct. Suite 110 NULL Fannin County GA NULL United States -5.00 condo 1676.20 +AAAAAAAABJHJAAAA Sir James Culpepper 16 Park Way Suite 460 Freeman Charlton County GA 32297 United States -5.00 single family 2096.68 +AAAAAAAABKKAAAAA Sir Jeffrey Young 379 7th Lane Suite 90 Plainville Brantley County GA 36115 United States -5.00 apartment 2161.20 +AAAAAAAABLIBAAAA Dr. Charles Hawkins 939 Jackson Forest Boulevard Suite Y Bunker Hill Candler County GA 30150 United States -5.00 condo 4396.68 +AAAAAAAABLPJAAAA NULL NULL Gibson 458 Second Boulevard Suite 480 Plainview Liberty County GA 33683 United States -5.00 single family 1513.94 +AAAAAAAABLPNAAAA Mr. Eric Durham 917 Second Park Wy Suite 180 Friendship Marion County GA 34536 United States -5.00 condo 2166.25 +AAAAAAAABMCOAAAA Sir Warren Skinner 934 Walnut Hickory Court Suite 310 Riverview Long County GA 39003 United States -5.00 condo 5699.34 +AAAAAAAABNJBAAAA Miss Santa Richmond 859 4th Maple ST Suite 60 Antioch Toombs County GA 38605 United States -5.00 single family 2136.92 +AAAAAAAABPGJAAAA Dr. June Hill 480 Second 15th ST Suite T Woodlawn Atkinson County GA 34098 United States -5.00 condo 3081.42 +AAAAAAAABPKOAAAA Mr. Robert Tanner 676 Sixth Walnut Road Suite L Cumberland Oglethorpe County GA 38971 United States -5.00 condo 3966.42 +AAAAAAAACAINAAAA Dr. Donald Beebe 960 Railroad Davis Ln Suite Y Silver Springs Clay County GA 34843 United States -5.00 apartment 4925.75 +AAAAAAAACBEPAAAA Sir Otis Hartman 939 Third Lane Suite R Forest Hills Polk County GA 39237 United States -5.00 single family 4904.39 +AAAAAAAACCBGBAAA Ms. Judy Williams 283 Center Cir. Suite Q Farmington Dade County GA 39145 United States -5.00 single family 3655.41 +AAAAAAAACCOFAAAA Sir John Bower 970 Johnson Franklin ST Suite 20 Walnut Grove Union County GA 37752 United States -5.00 single family 5456.48 +AAAAAAAACDNAAAAA Miss Beatriz Rome 947 Elm Ln Suite 210 Wilson Lanier County GA 36971 United States -5.00 apartment 3994.62 +AAAAAAAACEKEBAAA Mr. Gregory Maynard 214 Park Forest Wy Suite 90 Lakewood Jones County GA 38877 United States -5.00 apartment 3840.02 +AAAAAAAACGIBBAAA Dr. Pamela Taylor 364 Cherry Pkwy Suite 430 Rolling Hills Glascock County GA 37272 United States -5.00 apartment 2025.02 +AAAAAAAACIDCBAAA Dr. Andre Worth 307 10th 6th Parkway Suite I Five Forks Haralson County GA 32293 United States -5.00 single family 1947.99 +AAAAAAAACJLAAAAA Sir Kevin Ellis 291 7th Valley Circle Suite H Hartland Marion County GA 36594 United States -5.00 apartment 5655.03 +AAAAAAAACKDGAAAA Ms. Catherine Anderson 875 Walnut Lake Ct. Suite 410 Taft Telfair County GA 30589 United States -5.00 condo 1701.58 +AAAAAAAACKIOAAAA Mr. Francisco Davis 394 Spruce Dr. Suite 250 Smith Warren County GA 37317 United States -5.00 condo 4471.40 +AAAAAAAACNIEBAAA Dr. Andrea Lyon 700 Fifth Center Cir. Suite P Franklin Putnam County GA 39101 United States -5.00 condo 1725.43 +AAAAAAAACNOHAAAA Ms. Jessica Pitman 11 Spring East Street Suite 470 Glenwood Columbia County GA 33511 United States -5.00 condo 3469.20 +AAAAAAAACOOKAAAA Miss Rebecca Morris NULL Park Pine NULL NULL NULL NULL GA NULL United States -5.00 NULL 1834.52 +AAAAAAAADAFIBAAA Dr. Todd Alexander 371 East Pkwy Suite Y Jackson Long County GA 39583 United States -5.00 single family 3081.99 +AAAAAAAADBAABAAA Dr. Martha Valenti 605 13th Lake Ave Suite 100 Mount Pleasant Pickens County GA 31933 United States -5.00 single family 2721.38 +AAAAAAAADBNCAAAA Mrs. Katie Novak 352 Jackson Hill Street Suite 360 Liberty Jeff Davis County GA 33451 United States -5.00 apartment 2970.22 +AAAAAAAADCCHBAAA Dr. Harley Kohl 116 Center Park Cir. Suite 300 White Oak Cook County GA 36668 United States -5.00 single family 5818.25 +AAAAAAAADCICAAAA Mrs. Bonnie Harden 375 Railroad 3rd Road Suite A Greenwood Glynn County GA 38828 United States -5.00 single family 2770.41 +AAAAAAAADDJPAAAA Sir Rick Lindstrom 994 Jefferson View Road Suite R Spring Valley Barrow County GA 36060 United States -5.00 apartment 8135.54 +AAAAAAAADDKOAAAA Sir Derek Watkins 344 View Cedar Boulevard Suite 280 Fairfield Gordon County GA 36192 United States -5.00 single family 1699.26 +AAAAAAAADDLMAAAA Mr. Alan Meredith 994 Jefferson View Road Suite R Spring Valley Barrow County GA 36060 United States -5.00 apartment 2784.48 +AAAAAAAADGDBBAAA Sir Walter Barron 728 River Court Suite 110 Walnut Grove Paulding County GA 37752 United States -5.00 condo 4128.19 +AAAAAAAADGGCBAAA Ms. Anna Maynard 956 Hill Blvd Suite N Macedonia Emanuel County GA 31087 United States -5.00 condo 6425.51 +AAAAAAAADGPLAAAA Mrs. Colleen Rogers 633 6th Dr. Suite I Bridgeport Harris County GA 35817 United States -5.00 apartment 3184.68 +AAAAAAAADHBGAAAA Miss NULL NULL 862 Locust Circle Suite 430 Greenfield Liberty County GA 35038 United States -5.00 single family 14758.14 +AAAAAAAADJICBAAA Dr. Julian Olivares 56 1st Washington Parkway Suite 190 Friendship Jeff Davis County GA 34536 United States -5.00 single family 3078.43 +AAAAAAAADKOLAAAA Dr. Matthew Goodman 385 Oak RD Suite D New Hope Jefferson County GA 39431 United States -5.00 apartment 4047.82 +AAAAAAAADLEHAAAA Dr. Robert Lilley 926 Davis First Road Suite 360 Salem Chattooga County GA 38048 United States -5.00 apartment 2256.31 +AAAAAAAADLEHAAAA Dr. Robert Lilley 926 Davis First Road Suite 360 Salem Chattooga County GA 38048 United States -5.00 apartment 2298.55 +AAAAAAAADMOOAAAA Sir Eddie Craig 788 Lake Dr. Suite F Buena Vista Marion County GA 35752 United States -5.00 single family 5585.06 +AAAAAAAADNCKAAAA Dr. Scott Whitehurst 481 4th Wy Suite U Sugar Hill Effingham County GA 35114 United States -5.00 condo 3355.64 +AAAAAAAADNHFBAAA Dr. Gerald Hayden 884 1st Smith Court Suite 390 Highland Towns County GA 39454 United States -5.00 single family 3481.60 +AAAAAAAADONFBAAA Dr. Jane Williams 527 Cedar South Street Suite 310 Fairview Twiggs County GA 35709 United States -5.00 condo 2304.16 +AAAAAAAADPLEAAAA Mr. Michael Powell 616 6th Pkwy Suite G Glendale Dougherty County GA 33951 United States -5.00 single family 2061.25 +AAAAAAAAECBEAAAA Ms. Megan Wilson 93 13th Drive Suite X Riverdale Clarke County GA 39391 United States -5.00 single family 5256.96 +AAAAAAAAECBMAAAA Mrs. Paula Clark 145 Locust Drive Suite M Mount Vernon Wilcox County GA 38482 United States -5.00 condo 3445.05 +AAAAAAAAECIEAAAA Miss Danille Sanders 825 Jackson South Boulevard Suite 480 Riverdale Clarke County GA 39391 United States -5.00 single family 2897.72 +AAAAAAAAEDDFAAAA Dr. Michelle Louis 375 Highland East Avenue Suite L Oakdale Haralson County GA 39584 United States -5.00 apartment 2301.53 +AAAAAAAAEGJBAAAA Mrs. Manuela Roche 935 10th Davis Way Suite O Oakwood Worth County GA 30169 United States -5.00 apartment 6416.92 +AAAAAAAAEGMCBAAA Mr. Christopher Saunders 476 West Lane Suite J Plainview Gilmer County GA 33683 United States -5.00 single family 3623.76 +AAAAAAAAEGOCBAAA Mrs. Elise Huff 40 Central Court Suite P Farmington Coffee County GA 39145 United States -5.00 condo 2518.63 +AAAAAAAAEGOMAAAA Sir James Handy 316 Washington Lane Suite 480 Oakland Richmond County GA 39843 United States -5.00 single family 3035.76 +AAAAAAAAEHIJAAAA NULL NULL Boggs 721 2nd 5th Ave Suite C Greenwood Wilkes County GA 38828 United States -5.00 apartment 2623.44 +AAAAAAAAEICDBAAA Ms. Alice Garrett 157 West Smith Drive Suite 20 River Oaks Hancock County GA 38075 United States -5.00 single family 16853.44 +AAAAAAAAEIINAAAA Mrs. Marie Coffey 539 Hill Walnut Road Suite F Marion Dade County GA 30399 United States -5.00 apartment 3608.02 +AAAAAAAAEIINAAAA Mrs. Marie Coffey 539 Hill Walnut Road Suite F Marion Dade County GA 30399 United States -5.00 apartment 7928.23 +AAAAAAAAEJCKAAAA Mr. Bryan Simpson 313 Seventh 7th Ln Suite B Belmont Washington County GA 30191 United States -5.00 condo 2094.14 +AAAAAAAAELCCBAAA Ms. Robyn Martinez 274 Willow Birch Parkway Suite O Pleasant Valley Burke County GA 32477 United States -5.00 condo 3081.88 +AAAAAAAAEMFEAAAA Sir Dan Porter 110 North West Ave Suite 270 Farmington Calhoun County GA 39145 United States -5.00 single family 5717.10 +AAAAAAAAENCEBAAA Mrs. Latonya Doyle 634 Railroad Park Ln Suite S Oak Hill Dade County GA 37838 United States -5.00 single family 4450.56 +AAAAAAAAEPHFBAAA Mrs. Charlotte Matthews 368 Center 4th ST Suite L Martinsville Baldwin County GA 30419 United States -5.00 apartment 2301.95 +AAAAAAAAFACDAAAA Mr. William Marquez 724 1st Walnut Dr. Suite X Oak Hill Appling County GA 37838 United States -5.00 apartment 5728.95 +AAAAAAAAFADGAAAA Ms. Edna Brown 871 11th Third Cir. Suite 380 Waterloo Quitman County GA 31675 United States -5.00 condo 3552.69 +AAAAAAAAFANIAAAA Dr. Jeanette Mccormick 89 Jackson Birch RD Suite 440 Lakeview Coweta County GA 38579 United States -5.00 condo 9022.24 +AAAAAAAAFCPCAAAA Dr. Christine Smith 820 Park Lincoln Cir. Suite 60 Springfield Irwin County GA 39303 United States -5.00 apartment 1916.99 +AAAAAAAAFFAPAAAA Sir Donald Wright 35 Jefferson Parkway Suite A Buena Vista Marion County GA 35752 United States -5.00 single family 3434.76 +AAAAAAAAFFFDAAAA Mr. Nathan Pratt 317 Main Second Dr. Suite J Shaw Madison County GA 30618 United States -5.00 condo 3953.10 +AAAAAAAAFGJHAAAA NULL NULL Carey 605 13th Lake Ave Suite 100 Mount Pleasant Pickens County GA 31933 United States -5.00 single family 3349.63 +AAAAAAAAFGMKAAAA Mrs. Albert Bradshaw 774 15th 9th Blvd Suite I White Oak Peach County GA 36668 United States -5.00 single family 1695.86 +AAAAAAAAFKDJAAAA NULL NULL NULL 186 Spring Road Suite 290 Buena Vista Johnson County GA 35752 United States -5.00 condo 1919.82 +AAAAAAAAFLFNAAAA Mrs. Scarlet Cook 127 2nd Blvd Suite U Lebanon Columbia County GA 32898 United States -5.00 condo 2722.78 +AAAAAAAAFLHCAAAA Sir Patrick Taylor 635 Third Ave Suite H Kingston Wayne County GA 34975 United States -5.00 apartment 3413.14 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q82.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q82.sql.out new file mode 100644 index 0000000000000..9f526a8be98ca --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q82.sql.out @@ -0,0 +1,9 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAEIJBAAAA New games used to suggest. Annual, legal expenses see male pubs; almost early offences must come from the heads. En route small conditions underestimate e 64.55 +AAAAAAAAELDEAAAA Methods get hours. American, great schools l 89.24 +AAAAAAAAMOCDAAAA Colours think. Partial, rich things would not appeal extremely open students. New, working magis 66.03 +AAAAAAAAOIOBAAAA Just open variables used to mind well also new 65.54 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q83.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q83.sql.out new file mode 100644 index 0000000000000..2ccae595d52f2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q83.sql.out @@ -0,0 +1,32 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAEFEAAAA 35 9.04392764857881 4 1.03359173126615 90 23.25581395348837 43.000000 +AAAAAAAAAFDDAAAA 11 8.94308943089431 21 17.073170731707318 9 7.317073170731707 13.666667 +AAAAAAAAAFKBAAAA 2 1.5151515151515151 3 2.2727272727272725 39 29.545454545454547 14.666667 +AAAAAAAABGOBAAAA 37 17.129629629629626 31 14.351851851851851 4 1.8518518518518516 24.000000 +AAAAAAAACIEAAAAA 34 26.35658914728682 6 4.651162790697675 3 2.3255813953488373 14.333333 +AAAAAAAACLLBAAAA 63 23.863636363636363 16 6.0606060606060606 9 3.4090909090909096 29.333333 +AAAAAAAADHDAAAAA 4 8.88888888888889 2 4.444444444444445 9 20.0 5.000000 +AAAAAAAADKNCAAAA 6 NULL 41 NULL NULL NULL NULL +AAAAAAAAEIABAAAA 56 13.82716049382716 27 6.666666666666667 52 12.839506172839506 45.000000 +AAAAAAAAFKICAAAA 73 15.904139433551197 29 6.318082788671024 51 11.11111111111111 51.000000 +AAAAAAAAGDHAAAAA 79 21.236559139784948 3 0.8064516129032258 42 11.29032258064516 41.333333 +AAAAAAAAGJEAAAAA 94 23.55889724310777 4 1.0025062656641603 35 8.771929824561402 44.333333 +AAAAAAAAHKEBAAAA 12 7.4074074074074066 10 6.172839506172839 32 19.753086419753085 18.000000 +AAAAAAAAICPBAAAA 19 13.768115942028986 20 14.492753623188406 7 5.072463768115942 15.333333 +AAAAAAAAJICAAAAA 15 5.319148936170213 49 17.375886524822697 30 10.638297872340425 31.333333 +AAAAAAAAKDMBAAAA 7 11.666666666666666 9 15.0 4 6.666666666666667 6.666667 +AAAAAAAAKEEBAAAA 18 6.382978723404255 29 10.28368794326241 47 16.666666666666664 31.333333 +AAAAAAAALJIAAAAA 1 0.32362459546925565 9 2.912621359223301 93 30.09708737864078 34.333333 +AAAAAAAALLJAAAAA 53 16.358024691358025 25 7.716049382716049 30 9.25925925925926 36.000000 +AAAAAAAALMDCAAAA 103 21.458333333333336 46 9.583333333333332 11 2.291666666666667 53.333333 +AAAAAAAALNDBAAAA 31 10.99290780141844 29 10.28368794326241 34 12.056737588652483 31.333333 +AAAAAAAAMLDDAAAA 1 0.32051282051282054 22 7.051282051282051 81 25.961538461538463 34.666667 +AAAAAAAANAFEAAAA 12 10.0 21 17.5 7 5.833333333333333 13.333333 +AAAAAAAANFNAAAAA 10 2.849002849002849 82 23.36182336182336 25 7.122507122507122 39.000000 +AAAAAAAANIDEAAAA 18 5.172413793103448 14 4.022988505747127 84 24.137931034482758 38.666667 +AAAAAAAAPDAEAAAA 61 29.04761904761905 7 3.3333333333333335 2 0.9523809523809523 23.333333 +AAAAAAAAPDFCAAAA 7 4.861111111111112 20 13.88888888888889 21 14.583333333333334 16.000000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q84.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q84.sql.out new file mode 100644 index 0000000000000..4f956713021d5 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q84.sql.out @@ -0,0 +1,21 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAAKAAAA Perry , Lori +AAAAAAAAANFABAAA Harris , Alexander +AAAAAAAACHLDAAAA Phipps , Charles +AAAAAAAACIHPAAAA Nelson , Timothy +AAAAAAAACJOJAAAA Mata , Eva +AAAAAAAADEANAAAA Gaines , David +AAAAAAAAEJPOAAAA Wesley , Paul +AAAAAAAAFFKBAAAA Duncan , Diana +AAAAAAAAFOBCAAAA Null , Thomas +AAAAAAAAJFBBAAAA Jacob , Ross +AAAAAAAAJGFBBAAA Johnson , Windy +AAAAAAAALBGPAAAA Porter , Veronica +AAAAAAAANJBHAAAA Thompson , Kenneth +AAAAAAAANOLKAAAA Thomas , Billy +AAAAAAAAONJOAAAA Floyd , Janette +AAAAAAAAONLJAAAA Howard , Tommy diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q85.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q85.sql.out new file mode 100644 index 0000000000000..d46c5afed2b00 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q85.sql.out @@ -0,0 +1,8 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Not the product that 10.0 177.330000 86.330000 +Not working any more 38.0 892.470000 14.940000 +reason 25 79.0 4640.200000 47.240000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q86.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q86.sql.out new file mode 100644 index 0000000000000..4714aa00b0bfc --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q86.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +329088997.80 NULL NULL 2 1 +35430664.91 Books NULL 1 1 +34918777.34 Men NULL 1 2 +33760378.41 Children NULL 1 3 +33243530.17 Electronics NULL 1 4 +33145164.93 Home NULL 1 5 +32756827.42 Shoes NULL 1 6 +32285995.12 Music NULL 1 7 +31712254.78 Sports NULL 1 8 +30795989.18 Women NULL 1 9 +30076030.73 Jewelry NULL 1 10 +963384.81 NULL NULL 1 11 +615989.10 NULL NULL 0 1 +65752.25 NULL dresses 0 2 +54445.48 NULL lighting 0 3 +40391.44 NULL infants 0 4 +35471.06 NULL mens 0 5 +34067.03 NULL country 0 6 +34046.70 NULL accessories 0 7 +27537.73 NULL baseball 0 8 +26174.29 NULL guns 0 9 +17743.12 NULL athletic 0 10 +11766.61 NULL shirts 0 11 +2662546.58 Books computers 0 1 +2495626.03 Books parenting 0 2 +2425647.20 Books fiction 0 3 +2401196.81 Books sports 0 4 +2346704.09 Books history 0 5 +2302719.16 Books science 0 6 +2276270.78 Books self-help 0 7 +2257444.51 Books business 0 8 +2216374.08 Books home repair 0 9 +2206720.90 Books reference 0 10 +2149203.74 Books mystery 0 11 +2140640.32 Books travel 0 12 +2048647.89 Books romance 0 13 +1956865.76 Books cooking 0 14 +1859356.14 Books arts 0 15 +1617536.06 Books entertainments 0 16 +67164.86 Books NULL 0 17 +8836884.43 Children school-uniforms 0 1 +8760984.73 Children infants 0 2 +8142524.86 Children newborn 0 3 +7925776.25 Children toddlers 0 4 +94208.14 Children NULL 0 5 +2489469.99 Electronics disk drives 0 1 +2323648.66 Electronics personal 0 2 +2305415.91 Electronics camcorders 0 3 +2240199.44 Electronics karoke 0 4 +2235691.33 Electronics stereo 0 5 +2233058.51 Electronics televisions 0 6 +2113792.92 Electronics portable 0 7 +2092500.41 Electronics musical 0 8 +2088701.69 Electronics dvd/vcr players 0 9 +2081097.01 Electronics cameras 0 10 +2055973.50 Electronics automotive 0 11 +2021900.56 Electronics audio 0 12 +2006817.42 Electronics memory 0 13 +1729410.62 Electronics monitors 0 14 +1621759.97 Electronics wireless 0 15 +1604092.23 Electronics scanners 0 16 +2580794.04 Home mattresses 0 1 +2552960.47 Home glassware 0 2 +2479089.66 Home lighting 0 3 +2241963.26 Home bathroom 0 4 +2227778.30 Home bedding 0 5 +2201961.72 Home wallpaper 0 6 +2125093.38 Home rugs 0 7 +2029622.97 Home accent 0 8 +2005244.71 Home paint 0 9 +2004693.90 Home tables 0 10 +1882095.92 Home blinds/shades 0 11 +1833537.38 Home decor 0 12 +1805505.20 Home kids 0 13 +1761357.98 Home curtains/drapes 0 14 +1684930.79 Home flatware 0 15 +1596468.10 Home furniture 0 16 +132067.15 Home NULL 0 17 +2311781.76 Jewelry custom 0 1 +2143571.63 Jewelry birdal 0 2 +2116270.85 Jewelry loose stones 0 3 +2091585.73 Jewelry diamonds 0 4 +1929363.68 Jewelry bracelets 0 5 +1896051.77 Jewelry gold 0 6 +1858969.13 Jewelry costume 0 7 +1849246.62 Jewelry mens watch 0 8 +1845580.36 Jewelry estate 0 9 +1834863.97 Jewelry earings 0 10 +1809049.82 Jewelry womens watch 0 11 +1738077.18 Jewelry semi-precious 0 12 +1736010.32 Jewelry rings 0 13 +1703934.55 Jewelry consignment 0 14 +1608464.69 Jewelry pendants 0 15 +1603208.67 Jewelry jewelry boxes 0 16 +9509484.62 Men sports-apparel 0 1 +8897383.77 Men pants 0 2 +8508441.20 Men accessories 0 3 +7952307.23 Men shirts 0 4 +51160.52 Men NULL 0 5 +8668502.54 Music rock 0 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q87.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q87.sql.out new file mode 100644 index 0000000000000..cc6cf7d448318 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q87.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +47170 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q88.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q88.sql.out new file mode 100644 index 0000000000000..f2f9cd0812e84 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q88.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +2358 4664 4828 7447 6997 3886 4073 4923 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q89.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q89.sql.out new file mode 100644 index 0000000000000..7a48b3b50f536 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q89.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Men shirts importoimporto #x bar Unknown 3 1815.92 4915.190000 +Women dresses amalgamalg #x ation Unknown 6 1508.07 4586.300000 +Men shirts importoimporto #x ese Unknown 6 1577.69 4596.057500 +Men shirts importoimporto #x ation Unknown 6 2077.36 4920.419167 +Men shirts importoimporto #x ese Unknown 4 1762.17 4596.057500 +Men shirts importoimporto #x ation Unknown 3 2183.89 4920.419167 +Women dresses amalgamalg #x ation Unknown 3 1917.37 4586.300000 +Men shirts importoimporto #x ought Unknown 4 2425.64 5071.261667 +Men shirts importoimporto #x ought Unknown 2 2438.50 5071.261667 +Women dresses amalgamalg #x eing Unknown 7 1904.98 4494.382500 +Women dresses amalgamalg #x able Unknown 4 1994.71 4582.713333 +Men shirts importoimporto #x eing Unknown 3 2458.87 5035.925833 +Women dresses amalgamalg #x ought Unknown 3 2138.31 4707.378333 +Men shirts importoimporto #x ought Unknown 7 2511.79 5071.261667 +Men shirts importoimporto #x eing Unknown 4 2528.19 5035.925833 +Men shirts importoimporto #x ought Unknown 5 2641.17 5071.261667 +Men shirts importoimporto #x able Unknown 7 2360.38 4779.735000 +Men shirts importoimporto #x bar Unknown 6 2521.43 4915.190000 +Men shirts importoimporto #x ought Unknown 3 2696.60 5071.261667 +Men shirts importoimporto #x eing Unknown 7 2689.88 5035.925833 +Women dresses amalgamalg #x ought Unknown 4 2396.08 4707.378333 +Men shirts importoimporto #x eing Unknown 6 2725.29 5035.925833 +Women dresses amalgamalg #x able Unknown 7 2299.82 4582.713333 +Women dresses amalgamalg #x able Unknown 5 2321.48 4582.713333 +Women dresses amalgamalg #x ought Unknown 7 2449.19 4707.378333 +Men shirts importoimporto #x eing Unknown 2 2784.04 5035.925833 +Women dresses amalgamalg #x ese Unknown 4 2254.90 4490.582500 +Women dresses amalgamalg #x ese Unknown 3 2257.13 4490.582500 +Women dresses amalgamalg #x ese Unknown 6 2257.18 4490.582500 +Women dresses amalgamalg #x bar Unknown 4 1998.31 4205.080833 +Men shirts importoimporto #x ese Unknown 2 2393.90 4596.057500 +Women dresses amalgamalg #x eing Unknown 6 2308.58 4494.382500 +Men shirts importoimporto #x eing Unknown 1 2856.92 5035.925833 +Women dresses amalgamalg #x able Unknown 3 2408.40 4582.713333 +Women dresses amalgamalg #x ese Unknown 5 2317.37 4490.582500 +Men shirts importoimporto #x ese Unknown 7 2439.68 4596.057500 +Women dresses amalgamalg #x ese Unknown 7 2341.06 4490.582500 +Men shirts importoimporto #x ese Unknown 5 2457.43 4596.057500 +Women dresses amalgamalg #x ought Unknown 6 2577.66 4707.378333 +Women dresses amalgamalg #x bar Unknown 3 2097.37 4205.080833 +Women dresses amalgamalg #x eing Unknown 5 2398.63 4494.382500 +Men shirts importoimporto #x ation Unknown 4 2841.78 4920.419167 +Men shirts importoimporto #x bar Unknown 2 2884.81 4915.190000 +Women dresses amalgamalg #x ation Unknown 4 2557.46 4586.300000 +Men shirts importoimporto #x bar Unknown 1 2926.69 4915.190000 +Women dresses amalgamalg #x eing Unknown 3 2528.38 4494.382500 +Women dresses amalgamalg #x able Unknown 2 2616.82 4582.713333 +Men shirts importoimporto #x bar Unknown 4 2956.00 4915.190000 +Men shirts importoimporto #x able Unknown 5 2829.72 4779.735000 +Women dresses amalgamalg #x ese Unknown 2 2586.26 4490.582500 +Men shirts importoimporto #x ese Unknown 3 2728.14 4596.057500 +Men shirts importoimporto #x able Unknown 2 2913.44 4779.735000 +Men shirts importoimporto #x ought Unknown 6 3210.96 5071.261667 +Women dresses amalgamalg #x eing Unknown 4 2647.58 4494.382500 +Women dresses amalgamalg #x bar Unknown 7 2407.85 4205.080833 +Women dresses amalgamalg #x bar Unknown 6 2427.96 4205.080833 +Women dresses amalgamalg #x ought Unknown 2 2933.33 4707.378333 +Women dresses amalgamalg #x ought Unknown 5 2946.43 4707.378333 +Women dresses amalgamalg #x bar Unknown 2 2465.92 4205.080833 +Men shirts importoimporto #x able Unknown 4 3120.21 4779.735000 +Men shirts importoimporto #x able Unknown 7 868.21 2511.135833 +Men shirts importoimporto #x ese Unknown 6 985.62 2603.736667 +Women dresses amalgamalg #x bar Unknown 5 2594.86 4205.080833 +Women dresses amalgamalg #x ation Unknown 5 2992.12 4586.300000 +Men shirts importoimporto #x able Unknown 3 3197.55 4779.735000 +Men shirts importoimporto #x ation Unknown 7 3347.44 4920.419167 +Women dresses amalgamalg #x ation Unknown 7 3059.37 4586.300000 +Men shirts importoimporto #x ation Unknown 5 3402.55 4920.419167 +Women dresses amalgamalg #x able Unknown 6 3085.16 4582.713333 +Men shirts importoimporto #x bar Unknown 7 3426.04 4915.190000 +Women dresses amalgamalg #x eing Unknown 2 3013.27 4494.382500 +Women dresses amalgamalg #x bar Unknown 2 512.15 1992.243333 +Men shirts importoimporto #x ese Unknown 2 1143.39 2603.736667 +Women dresses amalgamalg #x able Unknown 2 652.58 2089.189167 +Women dresses amalgamalg #x able Unknown 3 656.84 2089.189167 +Women dresses amalgamalg #x bar Unknown 3 564.90 1992.243333 +Men shirts importoimporto #x bar Unknown 5 3491.77 4915.190000 +Men shirts importoimporto #x ation Unknown 3 1075.39 2478.654167 +Women dresses amalgamalg #x ese Unknown 1 3107.05 4490.582500 +Men shirts importoimporto #x ought Unknown 2 1061.74 2425.509167 +Men shirts importoimporto #x eing Unknown 7 1040.11 2396.908333 +Men shirts importoimporto #x ese Unknown 1 3253.77 4596.057500 +Men shirts importoimporto #x able Unknown 2 1181.12 2511.135833 +Men shirts importoimporto #x ation Unknown 2 3598.63 4920.419167 +Women dresses amalgamalg #x eing Unknown 3 758.25 2074.660833 +Women dresses amalgamalg #x ese Unknown 5 782.25 2094.761667 +Men shirts importoimporto #x ese Unknown 4 1293.95 2603.736667 +Women dresses amalgamalg #x ation Unknown 2 3282.12 4586.300000 +Men shirts importoimporto #x able Unknown 3 1216.42 2511.135833 +Men shirts importoimporto #x able Unknown 6 3503.61 4779.735000 +Men shirts importoimporto #x ese Unknown 1 1333.77 2603.736667 +Men shirts importoimporto #x able Unknown 6 1243.07 2511.135833 +Women dresses amalgamalg #x ought Unknown 2 760.60 2026.577500 +Men shirts importoimporto #x bar Unknown 4 970.79 2234.079167 +Women dresses amalgamalg #x ese Unknown 4 832.51 2094.761667 +Men shirts importoimporto #x ation Unknown 6 1226.01 2478.654167 +Men shirts importoimporto #x ought Unknown 6 1205.46 2425.509167 +Women dresses amalgamalg #x ation Unknown 4 595.85 1797.562500 +Men shirts importoimporto #x bar Unknown 2 1036.00 2234.079167 +Women dresses amalgamalg #x eing Unknown 4 878.88 2074.660833 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q9.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q9.sql.out new file mode 100644 index 0000000000000..3331c24a2af4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q9.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +358.920754 1040.839473 1718.531826 2401.883865 3085.520204 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q90.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q90.sql.out new file mode 100644 index 0000000000000..96a51e152051f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q90.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +0.58933333333333333333 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q91.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q91.sql.out new file mode 100644 index 0000000000000..4f59414d82c51 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q91.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q92.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q92.sql.out new file mode 100644 index 0000000000000..e01ed10078cd6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q92.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +39037.04 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q93.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q93.sql.out new file mode 100644 index 0000000000000..c311fa1002618 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q93.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +108 NULL +331 NULL +533 NULL +564 NULL +1391 NULL +1680 NULL +2096 NULL +2534 NULL +2766 NULL +2857 NULL +3400 NULL +3565 NULL +4075 NULL +4516 NULL +4888 NULL +4980 NULL +5111 NULL +5626 NULL +5743 NULL +6078 NULL +6760 NULL +6910 NULL +7034 NULL +7770 NULL +7791 NULL +8140 NULL +8364 NULL +8725 NULL +8913 NULL +8938 NULL +9203 NULL +9220 NULL +9860 NULL +10269 NULL +11218 NULL +11625 NULL +12405 NULL +12935 NULL +12954 NULL +13050 NULL +13298 NULL +13626 NULL +13730 NULL +14105 NULL +14422 NULL +14587 NULL +14627 NULL +14734 NULL +14958 NULL +15599 NULL +16142 NULL +16759 NULL +16918 NULL +17479 NULL +18540 NULL +18710 NULL +19327 NULL +19514 NULL +19917 NULL +20571 NULL +21532 NULL +22571 NULL +23925 NULL +25168 NULL +25250 NULL +25414 NULL +26175 NULL +26407 NULL +27166 NULL +27240 NULL +28852 NULL +29635 NULL +30207 NULL +30278 NULL +30438 NULL +30445 NULL +31204 NULL +31911 NULL +31987 NULL +32147 NULL +32154 NULL +32169 NULL +32291 NULL +32639 NULL +32646 NULL +33232 NULL +33313 NULL +33872 NULL +33951 NULL +34860 NULL +35502 NULL +35684 NULL +35774 NULL +35779 NULL +35957 NULL +36221 NULL +36333 NULL +36556 NULL +37254 NULL +37411 NULL diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q94.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q94.sql.out new file mode 100644 index 0000000000000..0c727d0b36846 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q94.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +38 84308.56 21690.18 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q95.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q95.sql.out new file mode 100644 index 0000000000000..9558536825f1b --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q95.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +101 237942.25 -27171.32 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q96.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q96.sql.out new file mode 100644 index 0000000000000..574180c4d661f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q96.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +888 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q97.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q97.sql.out new file mode 100644 index 0000000000000..34652db5ee36f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q97.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +537833 285408 200 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q98.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q98.sql.out new file mode 100644 index 0000000000000..03160c0d31d44 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q98.sql.out @@ -0,0 +1,2534 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL Books NULL NULL 557.55 2.41577919115267179 +NULL Books NULL 6.35 361.02 1.56424464817493959 +Precisely elderly bodies Books arts 1.40 4303.08 1.75015577438987686 +Abilities could affect cruel parts. Predominantly other events telephone strong signs. Accurate mate Books arts 25.69 9236.96 3.75687156218529913 +Germa Books arts 5.82 3191.92 1.29822295179047002 +Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 2096.07 0.85251703756969175 +Small objects stop etc mediterranean patterns; liberal, free initiatives would not leave less clear british attitudes; good, blue relationships find softly very Books arts 58.41 5760.88 2.34307458786895754 +Newly national rights head curiously all electrical cells. Chinese, long values might not pull bad lines. High fun clothes ough Books arts 3.28 571.10 0.23227873122369528 +Forward psychological plants establish closely yet eastern changes. Likewise necessary techniques might drop. Pleasant operations like lonely things; dogs let regions. Forces might not result clearl Books arts 2.43 3457.82 1.40637023708618106 +Early, short v Books arts 75.57 850.08 0.34574593563060564 +Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 4798.20 1.95153179505784395 +Scientific, difficult polls would not achieve. Countries reach of course. Bad, new churches realize most english Books arts 3.98 17.50 0.00711762878027433 +United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 11646.43 4.73685516316858938 +Naturally new years put serious, negative vehicles. Fin Books arts 3.34 20302.86 8.25761260902173683 +Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Books arts 4.03 9410.73 3.82754758236520025 +Much new waters Books arts 1.85 2864.61 1.16509889030066491 +Tall, following actions keep widely willing, secondary groups. Heads could afford however; agricultural, square pri Books arts 9.99 1903.68 0.77426786036757875 +Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 7841.73 3.18940132200803357 +Powerful walls will find; there scottish decades must not Books arts 4.16 5934.47 2.41367739815283298 +Too executive doors progress mainly seemingly possible parts; hundreds stay virtually simple workers. Sola Books arts 34.32 10139.18 4.12382396436467639 +Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 10008.95 4.07085660459009779 +Aside legitimate decisions may not stand probably sexual g Books arts 3.88 874.84 0.35581636355058234 +Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 15380.51 6.25558632178840388 +Unlikely states take later in general extra inf Books arts 0.32 6478.12 2.63479162023261224 +Sometimes careful things state probably so Books arts 5.08 17118.92 6.96263529595507190 +Circumstances would not use. Principles seem writers. Times go from a hands. Members find grounds. Central, only teachers pursue properly into a p Books arts 5.95 7863.28 3.19816617344888566 +Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 6519.57 2.65165023240074772 +Systems cannot await regions. Home appropr Books arts 7.30 889.75 0.36188058327137607 +Relevant lips take so sure, manufacturing Books arts 8.80 932.33 0.37919879089789497 +Extra, primitive weeks look obviou Books arts 1.18 10006.38 4.06981132996350893 +More than key reasons should remain. Words used to offer slowly british Books arts 0.28 1075.86 0.43757554854548205 +Children may turn also above, historical aspects. Surveys migh Books arts 7.22 3864.60 1.57181646767132336 +Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 3061.36 1.24512137387317768 +Useful observers start often white colleagues; simple pro Books arts 3.47 724.40 0.29462915933889837 +Members should say earnings. Detailed departments would not move just at the hopes. Figures can take. Actually open houses want. Good teachers combine the Books arts 3.09 7805.88 3.17482035104958588 +Major, senior words afford economic libraries; successful seconds need outside. Clinical, new ideas put now red c Books arts 5.87 616.62 0.25079270048530027 +Likely states feel astonishingly working roads. Parents put so somewhere able policies. Others may rely shortly instead interesting bodies; bri Books arts 7.50 8829.29 3.59106334933647431 +Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 6344.62 2.58049428068023382 +Accurate years want then other organisations. Simple lines mean as well so red results. Orthodox, central scales will not in Books arts 7.69 3640.38 1.48062134052200283 +Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 12867.38 5.23344195512721440 +Weak effects set far in the effects. Positive, true classes sell frankly ever open studies. Unique problems must mean as yet new genes. Essential businesses agree deep current stages. Not Books arts 4.40 4471.87 1.81880632077973420 +Visitors could not allow glad wages. Communist, real figures used to apply factors. Aggressive, optimistic days must mean about trees. Detailed courts consider really large pro Books arts 9.08 1737.00 0.70647549664780021 +Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 8016.08 3.26031324074179520 +Similar situations come separate programmes. National, large others could not ask opportunities. Severe, large findings accept; twins go more. Tiny rights may see specifi Books arts 1.27 2413.11 0.98146406776958731 +Natural plans might not like n Books business 4.29 8813.76 2.98246237973343018 +Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 1861.77 0.62999888636816844 +Whole, important problems make. Indeed industrial members go skills. Soft Books business 3.22 6877.01 2.32709123121693768 +Cheap depths calm as in a traditi Books business 7.92 2554.82 0.86451804190159048 +Simple, great shops glance from a years. Lessons deepen here previous clients. Increased, silent flights open more great rocks. Brill Books business 8.92 8014.35 2.71195237594586375 +Just sudden ideas ought to serve full sources; uncertain, open qualifications shout questions; chronic, informal Books business 4.62 4172.49 1.41191664565564981 +Groups must not put new, civil moves. Correct men laugh slightly total novels. Relatively public girls set even scott Books business 3.36 414.96 0.14041709657333354 +Just young degrees stop posts. More than tight artists buy to a arts. European, essential techniques ought to sell to a offences. Sentences be Books business 2.58 6286.70 2.12733796276165399 +Other, black houses flow. New soldiers put only eastern hours. Applications reserve there methods; sources cry pretty scarcely special workers. Never british opportunities Books business 8.20 3426.05 1.15933100471146462 +Junior, severe restrictions ought to want principles. Sure, Books business 9.77 3899.69 1.31960465427044307 +Rows could not Books business 1.65 15875.48 5.37205708576254358 +Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 916.17 0.31002007751973922 +Manufacturing, ready concerns see already then new pupils. Both stable types used to manage otherw Books business 1.18 8723.00 2.95175036969632840 +Russian windows should see in a weapons. New, considerable branches walk. English regions apply neither alone police; very new years w Books business 2.79 8434.64 2.85417307557668685 +Long groups used to create more tiny feet; tools used to dare still Books business 57.04 795.66 0.26924105229308502 +Drugs must compensate dark, modest houses. Small pubs claim naturally accessible relationships. Distinguished Books business 1.66 11559.25 3.91150068335575881 +Small, capable centres Books business 2.98 19190.81 6.49392187461561344 +Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 1761.61 0.59610603791823330 +Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 3500.18 1.18441563785437289 +Sound, original activities consider quite to a attitudes. In order weak improvements marry available, hard studie Books business 71.27 11431.86 3.86839355512056274 +Always other hours used to use. Women should jump then. Civil samples take therefore other offices. Concrete, major demands Books business 1.42 2038.40 0.68976819369356825 +Changes ensure different clients. Distinct, alone attacks think directly previous feelings. White children tell so medieval, responsible yea Books business 5.89 5116.38 1.73131681262259552 +Visual fragments Books business 6.77 2739.02 0.92684893931051673 +Classic issues will draw as european, engl Books business 75.64 14735.99 4.98646840884344817 +Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 9396.38 3.17961345165736401 +Critical cases tell anywhere to the circumstances. Dependent, new numbers must not Books business 3.72 726.75 0.24592279963049486 +Confident, video-tape Books business 3.17 6173.95 2.08918482116091330 +Of course fundamental children will not deal still including a suppliers. More crucial powers will not keep enough. As good comments used to devote even convenient electric problems. Publi Books business 8.85 2672.80 0.90444094785330122 +Departments could seek now for a commu Books business 5.93 3205.71 1.08477079876638965 +Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 12939.97 4.37871847201185356 +Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 6877.89 2.32738901183430931 +So white countries could secure more angry items. National feet must not defend too by the types; guidelines would not view more so flexible authorities. Critics will handle closely lig Books business 2.50 2135.27 0.72254774869901171 +Simple changes ought to vote almost sudden techniques. Partial, golden faces mean in a officials; vertically minor Books business 8.73 5996.87 2.02926323965617573 +Christian lines stand once deep formal aspirations. National, fine islands play together with a patterns. New journals lose etc positive armie Books business 4.89 6106.50 2.06636061361350790 +Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 5521.91 1.86854291917113983 +Lips will n Books business 8.48 7806.43 2.64159493735051117 +White fees might combine reports. Tr Books business 2.09 6566.98 2.22218108939451963 +Asleep children invite more. Wealthy forms could expect as. Indeed statistical examinations could la Books business 3.71 11535.83 3.90357565828889099 +Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 9334.32 3.15861315039135987 +Local, bloody names Books business 4.40 2927.75 0.99071273012477651 +Large, larg Books business 3.50 5826.76 1.97170021599584758 +Only new systems might join late speeches. Materials could stay on a benefits. Corporate regulations must crawl definitely practical deaths. Windows might soothe despite a organisations. Old Books business 0.67 123.41 0.04176034771571981 +Professional managers form later initial grounds. Conscious, big risks restore. American, full rises say for a systems. Already Books business 5.27 1616.40 0.54696885218126163 +Memories can earn particularly over quick contexts; alone differences make separate years; irish men mea Books business 4.23 6855.84 2.31992757704675870 +Only, gothic Books business 1.68 7807.37 2.64191302119179451 +Low, large clouds will not visit for example as the notions. Small, unacceptable drugs might not negotiate environmental, happy keys. Books business 3.11 3933.56 1.33106582416859905 +Silver, critical operations could help howev Books business 5.56 9087.69 3.07515674850230731 +Terrible, psychiatric bones will destroy also used studies; solely usual windows should not make shares. Advances continue sufficiently. As key days might not use far artists. Offici Books business 5.83 2024.32 0.68500370381562209 +Too white addresses end by the talks. Hands get only companies. Statements know. Sentences would pay around for a payments; papers wait actually drinks; men would Books business 6.06 5178.86 1.75245923645598158 +New, big arguments may not win since by a tenant Books computers 1.00 1686.72 0.45326198032962534 +Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 471.00 0.12656895793922734 +Cheap, desirable members take immediate, estimated debts; months must track typica Books computers 3.26 7226.83 1.94202195818247621 +Expert, scottish terms will ask quiet demands; poor bits attempt northern, dangerous si Books computers 2.66 4463.91 1.19955931429829364 +Gradually serious visitors bear no doubt technical hearts. Critics continue earlier soviet, standard minute Books computers 6.40 3244.45 0.87186126451364360 +Clear, general goods must know never women. Communications meet about. Other rewards spot wide in a skills. Relative, empty drawings facilitate too rooms. Still asian police end speedily comp Books computers 7.64 5385.04 1.44708896233770017 +Wide, essential activities make steadily procedures. Modules Books computers 35.95 7285.54 1.95779873848101557 +At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 14393.77 3.86795004186180949 +Environmental, new women pay again fingers. Different, uncomfortable records miss far russian, dependent members. Enough double men will go here immediatel Books computers 89.89 9487.37 2.54948308807619376 +Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 1706.66 0.45862033493962150 +Magnetic Books computers 57.19 7638.33 2.05260184394042112 +Ga Books computers 5.53 7904.13 2.12402865714688954 +S Books computers 65.78 578.19 0.15537347301673430 +Simple year Books computers 3.01 5038.44 1.35394925783295241 +Agricultural players shall smoke. So full reasons undertake Books computers 0.70 5739.18 1.54225484506508439 +Then basic years can encourage later traditions. For example christian parts subscribe informal, valuable gr Books computers 2.75 11359.94 3.05268740563088364 +Boxes batt Books computers 0.83 6659.54 1.78957757569979198 +Blocks extend ev Books computers 9.29 11249.90 3.02311702743208836 +Separate, dead buildings think possibly english, net policies. Big divisions can use almost Books computers 9.46 3529.22 0.94838577014496795 +Artists make times. Rather ready functions must pre Books computers 5.71 7805.93 2.09763996995021836 +Limited, capable cities shall try during the bodies. Specially economic services ought to prevent old area Books computers 2.93 6458.26 1.73548882866368225 +Since other birds shall blame sudden Books computers 6.74 2404.97 0.64627292308939187 +Legs throw then. Old-fashioned develo Books computers 2.66 12518.22 3.36394492707854445 +Only careful men define judicial, special lawyers. Now able funds will not put too black, economic terms. Objectives know both points. Teeth pay. Books computers 9.85 911.50 0.24494183686115864 +Men should not turn shadows. Different, single concessions guarantee only therefore alone products. Books computers 8.38 11864.77 3.18834729318175442 +Educational, white teachers should not fix. Considerable, other services might not cover today on a forms. Successful genes fall otherwise so Books computers 1.65 7042.80 1.89256869845942737 +Women note days. Other, efficient qualificati Books computers 7.64 8012.26 2.15308577269247054 +Present Books computers 2.84 4786.32 1.28619858760866792 +Multiple, dark feet mean more complex girls; schools may not answer frequently blue assets. Spiritual, dry patients may reply personnel Books computers 2.04 2973.19 0.79896721880112808 +Private teachers ap Books computers 5.27 8109.35 2.17917617635769258 +Daily numbers sense interesting players. General advantages would speak here. Shelves shall know with the reductions. Again wrong mothers provide ways; as hot pr Books computers 7.56 13142.36 3.53166626340166409 +Inc, corporate ships slow evident degrees. Chosen, acute prices throw always. Budgets spend points. Commonly large events may mean. Bottles c Books computers 68.38 12405.10 3.33354687926095337 +European, possible problems ought to restore then unfair interests. States would sleep in respect of the questions. Ideal stages affect only pressures. About spanish employees might kno Books computers 3.42 6760.19 1.81662463645686890 +Upper others narrow deaths. Situations could e Books computers 5.42 10932.74 2.93788855460829783 +However old hours ma Books computers 8.84 5208.75 1.39971562561772907 +Indeed other actions should provide after a ideas; exhibitio Books computers 6.95 3949.76 1.06139491997885895 +Effective females must answer too english projects. Firm, political experiments see in terms of Books computers 0.76 246.87 0.06633986973770075 +Of course responsible fears tell. Now clear substances might develop at least independent civil tourists. Books computers 4.95 619.44 0.16645833398274943 +Perfect days find at all. Crimes might develop hopes. Much socialist grants drive current, useful walls. Emissions open naturally. Combinations shall not know. Tragic things shall not receive just Books computers 6.71 8038.78 2.16021233057898500 +Advantages apply almost on a services; materials defeat today individual ideas. Domestic divisions used to win smoothly irish talks. Subsequent quantities make only, automatic pounds. Flower Books computers 7.87 442.26 0.11884583298981461 +Close, historic tactics lead ago large, typical stars. Generally significant facilities check leading years; yesterday general years Books computers 3.87 8448.38 2.27028164092273769 +Internal seats used to sell dark words. Universal items show now in the roles. Most wonderf Books computers 2.57 870.24 0.23385428865612144 +Likely, separate attacks prefer seats. Informally equal women could use easy prime, big forces. Long technical women save conditions; fast alone rooms sell. Ne Books computers 3.77 344.40 0.09254851191989362 +Economic customs should not put unpleasant shops. Colonial, middle goods used to see. Closely explicit legs continue Books computers 3.32 8481.54 2.27919252551990282 +Human windows take right, variable steps. Years should buy often. Indeed thin figures may beat even up to a cars. Details may tell enough. Impossible, sufficient differences ought to return Books computers 4.47 5466.16 1.46888784528468556 +Left diff Books computers 0.74 3269.32 0.87854442796151745 +Male levels shall reduce else high, local conditions; further personal agencies control. Successful days wake eve Books computers 6.55 2376.38 0.63859010672531010 +Wide governments conform widely in proportion to a friends. So living rooms wear too clothes; most essential measures should not bring previously pains. Real accounts become also gue Books computers 9.35 11110.42 2.98563541755233586 +Places transform Books computers 3.10 5805.20 1.55999599708875273 +Appropriate effects beg passages. Running contracts must keep only upper sons. Safely available reports intend perhaps w Books computers 5.81 8969.60 2.41034591323077181 +Friendly, hot computers tax elsewhere units. New, real officials should l Books computers 3.19 2999.57 0.80605615534133364 +Perfect members state democratic schools. Genuine, enormous knees must afford around the implications. Matters will indicate with a months. Still regular machines would not l Books computers 4.07 2110.95 0.56726272136265806 +Kinds relieve really major practices. Then capable reserves could not approve foundations. Pos Books computers 7.23 1522.62 0.40916438797755059 +Only increased errors must submit as rich, main Books computers 6.94 8287.27 2.22698753303826016 +Meals ought to test. Round days might need most urban years. Political, english pages must see on a eyes. Only subsequent women may come better methods; difficult, social childr Books computers 7.23 15325.54 4.11833891222069241 +Systems cannot see fairly practitioners. Little ca Books computers 1.73 2428.71 0.65265242852777245 +Past beautiful others might not like more than legislative, small products. Close, wh Books computers 3.02 4174.86 1.12188467036552578 +Main problems wait properly. Everyday, foreign offenders can worry activities. Social, important shoes will afford okay physical parts. Very Books computers 1.40 939.26 0.25240161238640906 +Schools offer quickly others. Further main buildings satisfy sadly great, productive figures. Years contribute acti Books computers 4.11 885.92 0.23806787944271822 +Chief me Books computers 2.62 9675.59 2.60006230094948754 +Tiny, rare leaders mention old, precious areas; students will improve much multiple stars. Even confident solutions will include clearly single women. Please little rights will not mention harder com Books computers 1.45 3092.13 0.83092923972956056 +Guidelines should investigate so. Usual personnel look now old, modern aspects. Discussions could appear once br Books computers 2.44 9923.72 2.66674076280396839 +Flat pleasant groups would go private, redundant eyes. Main devic Books computers 2.83 2445.21 0.65708637291417851 +Popular, obvious copies should believe still difficult parts. Forms ought to soften characteristic Books computers 1.05 2156.19 0.57941979069847684 +Real, domestic facilities turn often guilty symptoms. Winds get naturally intense islands. Products shall not travel a little clear shares; improved children may not apply wrong c Books computers 5.28 1338.00 0.35955258115219995 +Directions would ask yet profits. Forthcoming, specified discussions ought Books cooking 0.58 5750.02 2.05632295473197783 +Then irish champions must look now forward good women. Future, big models sign. Then different o Books cooking 85.81 2279.71 0.81527020830049933 +Black ears see sensibly glad months. Equal members must afford approximately o Books cooking 8.37 10363.44 3.70617485886789408 +Considerable benefits should govern. Well experienced years provide please in an towns. Exc Books cooking 4.18 0.00 0.00000000000000000 +Valuable studies should persist so concerned parties. Always polite songs include then from the holes. There conventional areas might not explain theore Books cooking 1.58 1326.45 0.47436523408687831 +Meanings occur in a things. Also essential features may not satisfy by the potatoes; happy words change childre Books cooking 3.46 1262.55 0.45151330717055917 +Then dominant goods should combine probably american items. Important artists guess only sill Books cooking 6.67 5569.20 1.99165808110116677 +Libraries shall note still. Children would not concentrate. Local, public modes must mean low children. Outer, good years should vis Books cooking 1.42 2178.99 0.77925070784648269 +Children ought to miss historical effects. Honest girls may not think activities. Woo Books cooking 4.42 348.88 0.12476651428114901 +Single, past rates mark blue, evident discussions. Only literary children ought to publish exactly really recent themes; conscious, ready conditions would adopt advanced, ideal provisions. A Books cooking 4.99 9499.97 3.39738059698316657 +Standards could lead no longer ago great tactics; difficult lives might feel french, easy costs. Students drop certainly unabl Books cooking 3.05 16321.01 5.83672187356046718 +Individual, remarkable services take by the interest Books cooking 6.05 1054.65 0.37716408016112647 +Positions shall Books cooking 4.21 2629.53 0.94037288551281172 +Ultimately senior elections marry at l Books cooking 5.06 7756.87 2.77401293175881769 +Hence young effects shall not solve however months. In order small activities must not return almost national foods. International decades take contributions. Sessions must see Books cooking 1.43 19276.07 6.89351084309627374 +Members need for a regions. Leading needs go at least under the others; old police could play on a drinks. Very similar machines must consider fully effec Books cooking 9.86 10833.86 3.87440652490818908 +Mainly catholic activities could assume just fat, o Books cooking 2.68 2262.61 0.80915490391444210 +Points trace so simple eyes. Short advisers shall not say limitations. Keys stretch in full now blue wings. Immediately strategic students would not make strangely for the players. Books cooking 1.69 5132.94 1.83564271902740482 +Projects become more from a pupils. Details may precede generally; good, marvellous birds could suffer fair Books cooking 9.88 628.36 0.22471419087853357 +Great pp. will not r Books cooking 1.91 2941.23 1.05184308300603044 +New, general students raise therefore to a women; united letters would start black positio Books cooking 4.03 3747.49 1.34017789670793138 +Products may not resist further specif Books cooking 5.37 8721.33 3.11892325153523644 +Dramatic months deal broadly in a films. Almost new occasions may get together sources. Under dry orders wor Books cooking 3.92 1412.78 0.50523858073297895 +Thus certain stars appear totally even local guests. Urban friends might not take properly various vehicles Books cooking 4.55 1446.44 0.51727607462974425 +Somet Books cooking 7.34 6593.72 2.35804706645808830 +Genetic properties might describe therefore leaves; right other organisers must not talk even lives; methods carry thus wrong minutes. Proud worke Books cooking 1.08 119.92 0.04288580713309846 +Urgent agencies mean over as a plants; then Books cooking 6.47 9566.59 3.42120525067902230 +Men could require evolutionary falls; taxes teach dead parents; only financial servants might not buy eastern things. Different payments develop. New inhabitants might not eat w Books cooking 80.50 3855.42 1.37877583836799906 +Hours ought to cope thus into the eyes. Dark states reduce most for the feelings. National, tragic children shall establish enough typical boats. In order secret hours must mean; sin Books cooking 2.30 12966.63 4.63712802990534045 +Guests agree around trying, young costs. Here annual banks appeas Books cooking 58.88 8031.52 2.87223330308224573 +Wonderful qualities suffer of course light leaders. True clubs used to see early living operat Books cooking 9.91 4482.62 1.60307518988467144 +High big appeals may Books cooking 36.23 675.62 0.24161531867298181 +Final women should establish on a pupils. Full, northern years might not avoid full Books cooking 60.38 5877.02 2.10174071245298770 +Little part Books cooking 9.90 4729.36 1.69131438311366337 +Here other affairs afford directly effective leads. Plants cannot undertake as coming, huge photographs; d Books cooking 0.87 20785.39 7.43327407210001090 +Stairs might bring early others. Large forms rel Books cooking 1.88 2350.18 0.84047169953356678 +Now available m Books cooking 3.55 1102.96 0.39444070910208700 +Major instructions put flatly british, other democrats. Operations represent well upon a stores. Thousands will not appear surely Books cooking 1.29 582.88 0.20844962693245854 +New, single products raise too extreme, efficient minutes; hands support leaders. Additional, english things prefer halfway private, slow churches. High white things could f Books cooking 4.13 2472.08 0.88406559454294555 +Golden, sure days fill of course. Early free minutes must not express only, cap Books cooking 9.44 4521.21 1.61687575106934680 +Purposes hide tears. Small laws award good eyes. Books cooking 55.11 5382.78 1.92499053468895684 +Yet religious animals ensure also. Rough, real heads resist dead. Civil, evolutionary votes dissuade rapidly left cars. Buyers Books cooking 2.20 11624.81 4.15726617427380135 +Here comprehensive years should tend sensibly particular front sales. Official, coherent tears regulate animals. Rewards cannot w Books cooking 2.50 2499.59 0.89390372458156745 +Widely splendid others deprive only. Different, main soldiers discover then other periods. Too main birds must change public, terrible houses. Different, armed females may foster; science Books cooking 4.26 6853.89 2.45108909816104214 +New women add however. Scottish managers place mostly. Normal, financial purposes should lea Books cooking 4.74 319.20 0.11415234853973505 +Extra theories drop. Other resources shall know eventually anyway open students. Long-term, liable di Books cooking 6.96 5834.64 2.08658477093947276 +Special, public skills agree at a parent Books cooking 5.87 4713.66 1.68569974692295585 +Gentle fans cannot pay else can Books cooking 2.45 7576.48 2.70950183478800689 +Sound, new offices might equip hot, new reports; calculations fight great scientists. Professional, little issues learn of c Books cooking 66.16 6628.48 2.37047794250834265 +Well angry rebels drop in a methods. Studies argue most sometimes residential organisations. Rural, different children know o Books cooking 4.42 453.06 0.16202338041795852 +Half general features used to defend as ready medical pounds. Turkish, trying rooms secure with a ci Books cooking 7.08 683.53 0.24444409397670770 +African, elected carers would examine proba Books cooking 6.20 15598.69 5.57840569437117702 +Already accessible clubs match all enough o Books entertainments 5.00 1196.30 0.46493128593083651 +Likely, various days develop no longer. Officials say before agricultural, rare ho Books entertainments 2.67 23516.84 9.13960934734576042 +Less progressive experiences would silence as economic, soviet specialists. Alone legal brothers fight only ears. Methods could not return records. E Books entertainments 8.36 5931.28 2.30513887621487248 +Strict heads discuss as categories. Alone, specific markets wait single, human numbers. External, various changes want very relatively nuclear orders. Old, pre Books entertainments 2.32 4525.09 1.75863572068274594 +Instances used to lower out of a costs. Irish supporters sign in a types. Bad things shall participate clear Books entertainments 1.58 3570.57 1.38767006737947580 +Trustees may encourage today necessary, political tears; inner, foreign times pay in the historians. Areas may belie Books entertainments 1.79 17322.75 6.73233171726021741 +Rare, radical beds say over readers; han Books entertainments 7.10 7808.46 3.03468807902658165 +L Books entertainments 1.63 4264.23 1.65725481685601518 +Always constitutional advertisements go for a spaces. Cars spend bad difficulties. Rights encourage further great qualities. Blue, high homes would produce u Books entertainments 2.63 3974.52 1.54466161878945775 +Companies ought to record now detailed, good roads. Muscles shall not argue significantly territorial months. Clearly new periods could write in a committees. Figures will not find more there Books entertainments 3.07 7276.45 2.82792715498740725 +Falsely large trees shall reflect against a Books entertainments 0.70 957.09 0.37196446079707792 +Deep patterns shall find british, american expectations. Sufficient patients must see. English, large assets could not meet for the proceedings. White, chinese matches shal Books entertainments 0.56 1499.01 0.58257681762365897 +Particular, deliberate things rain however original ways. Virtually old deaths consider women. Notably w Books entertainments 9.71 1611.84 0.62642718708915783 +New, previous police outline right in a persons. Wealthy quest Books entertainments 2.07 5037.58 1.95781037146155928 +Doors cannot happen here severe, old rates. Inevitable, int Books entertainments 2.29 1047.84 0.40723363591888968 +Limitations respond. Bare rivers will not create yesterday. Well local persons may unders Books entertainments 8.95 2096.28 0.81470045646668390 +So perfect changes would twist again; legal standards like improvements; rights used to tell working stations. Official, immediate loans listen much possible pictures. Always d Books entertainments 6.32 1017.52 0.39545003933824690 +Prisons take angry, logical sums. Now old grounds cannot help so increased problems. Blue, negative designs would freeze. Small payments ask alike to a hundreds. Exte Books entertainments 2.62 11202.91 4.35391068500161131 +High, official employees shall not start too left circumstances. Patients used to touch obviously popular, senior members. British, impossible theories make only. Young, international wo Books entertainments 4.85 1041.70 0.40484737988309988 +Now old tears give. Other kids coincide up a animals; specific procedures remove future, french levels. Coming, strong values a Books entertainments 5.08 24460.84 9.50648649682223761 +Large women establish today polite, easy horses. Details sha Books entertainments 5.06 1748.58 0.67956996401650263 +Plans would not claim; most particular horses will not tell simply cases; more british enquiries could not smile blue men. Old, dangerous play Books entertainments 0.95 6942.27 2.69805108950854163 +Pieces threaten Books entertainments 0.69 1273.35 0.49487607869266126 +Cases can accept gmt sudden services; tools show all also capable meals; important, spatial days would not happen human, cold backs. Red, economic effects must s Books entertainments 9.58 1334.73 0.51873086622959576 +Financial gods might presume divine, tiny Books entertainments 8.42 731.84 0.28442306469583164 +Marginal, available teeth pay recently little services. Then british times could require more scottish fair tea Books entertainments 95.74 3018.65 1.17317130007115240 +Now complete others shall pass. Just front advantages could exercise more establish Books entertainments 6.51 5281.66 2.05266987849992639 +Young reasons could not say impossible experiences. Prisoners cancel particularly; forms might k Books entertainments 3.77 3626.88 1.40955444480216694 +Just particular actions seem very; necessarily poor eleme Books entertainments 0.26 6872.96 2.67111437845958545 +Japanese, efficient sports withdraw recently severe days; factors mean originally impossible items. Quiet workers become from a officers. Pieces explore. For example o Books entertainments 3.74 16796.75 6.52790652592057016 +Never able feet go on the provisions. Services play brown studies. Cruel, Books entertainments 9.79 12846.63 4.99272774870656373 +Internal claims speculate perhaps through a expectations. Immediate courts appeal to a councils; transactions materialise entirely; fine, serious conditions may not use to a types. Short, large lip Books entertainments 3.11 5231.34 2.03311346095579892 +Front, possible foundations hear well. Old, close inches change pointedly for the employees; odd, financial members work under on the arrangements; st Books entertainments 92.23 225.66 0.08770073893099771 +Local words co Books entertainments 2.95 9381.26 3.64594271959501737 +Hardly local women should tell easily tall, able issues. Important, available conditions could no Books entertainments 2.21 15740.54 6.11741996442846214 +General, raw tests would not buy heavy, considerable blues. High, regional modules meet often young, responsible calculations. Things hesitat Books entertainments 2.00 5567.90 2.16391449212931922 +H Books entertainments 4.80 2493.52 0.96908422644341674 +New hours borrow new poets. Youngsters mind especially. Laws must add there in a ends. Factors must not take strategic, royal tr Books entertainments 2.30 4109.90 1.59727584389128560 +Clear materials will ship evidently literally scottish targets. Residential heads make prominent times. Internal, open subjects feel subsequent Books entertainments 0.75 263.40 0.10236805208909332 +Other practices get feet. Numbers will not increase now large, simple foreigners. Flowers cover Books entertainments 1.00 315.51 0.12262013710945267 +Heavy, formal factors could want then truly serious players. Be Books entertainments 4.31 8757.62 3.40357061631163789 +Men call tonight particularly mental lines. Recent markets must dress children. Multiple relations should seem relatively about a arts. Funny, real proteins shall keep citie Books entertainments 5.20 3090.94 1.20126616144366780 +Dirty trials should get. Balls shall win later national programmes. Elements ought to explain apart poss Books entertainments 1.62 290.34 0.11283804192690719 +Subsequent, Books entertainments 1.29 9603.95 3.73248919461293761 +Countries turn more actually scientific patients. Good writers could not drag perhaps. Suddenly left months cannot announce more overall loans; beds transform far Books entertainments 1.32 2401.56 0.93334479565331415 +Royal, blue men used to convey jobs. Other, technical things would say as mere children; ab Books fiction 0.62 555.50 0.18274906106295868 +Exclusively ready fields invest right in a courts. Quite glad facts would not imitate usually by a types. More large managers can continue both small matters. Additional, basic scholars s Books fiction 1.11 3969.44 1.30587116641899316 +Dollars get on a years; separate economies can say. Firms know even sons. Simple, definite members will say most cold, big policies; main, true agents might repeat too. Elements know goods. Great Books fiction 5.03 149.04 0.04903135924540659 +Wild officials will not watch onl Books fiction 0.47 6954.51 2.28790310108543073 +Just minor eyes exc Books fiction 7.11 16681.12 5.48777500896227056 +Married circumstances face human, compulsory hours. Years make sometimes national problems. Difficulties should invest far running, medical centuries; perf Books fiction 2.71 10221.52 3.36268799754501063 +Other horses apply able schools; possible enquiries would not describe easily r Books fiction 3.83 10067.63 3.31206107944063852 +Firm, local examinations may not sponsor most rural charges. Countries shall help beautiful, different terms Books fiction 7.72 5090.34 1.67462620250444840 +As joint men would so Books fiction 2.13 2773.11 0.91230107781152357 +Pictures get with a conditions; still gross eyes go that. Personal beings contact thereafter in a systems. New, medium goals might not tell; as official years mu Books fiction 5.52 2061.58 0.67822107885899974 +Essential, alternative fans let unlikel Books fiction 1.52 2460.17 0.80934969856932323 +Basic changes may not see; afraid names seek in excess of a characteristics. Awful scientists shall not want now right eyes. Here used workers will not pray in part Books fiction 2.27 6034.24 1.98515156476786280 +Local companies would restrict yet most imaginative days. Married, str Books fiction 99.71 7003.69 2.30408239689654919 +Different stations may smell; weapons disguise cons Books fiction 1.47 1671.19 0.54979010505455611 +Private, quiet populations shall receive more somewhat proposed machines. Heads protect abroad parent Books fiction 74.86 3243.16 1.06693869464796593 +Circumstances should include parties. Good investigations fall as deposits. Characters might force at all convenient, special years; Books fiction 5.18 12.59 0.00414187340914968 +Old, official cases look enough; actual emotions go statistical, wild limits. Mental cities hear above mod Books fiction 2.55 769.44 0.25313130070978025 +Times should not get on a lists; different students undermine suddenly groups. Even actual modules may stay for a Books fiction 8.31 638.38 0.21001502358482729 +Techniques render eventually dark tiles. Only, other centres would bid at the falls. Sorry, full days write for a groups. Both Books fiction 2.99 6665.04 2.19267291079579140 +Towns see even afraid, mean factors. Soldiers spend areas; resu Books fiction 48.40 9444.91 3.10719790157362568 +Loud young standards remove enough green values; important students cannot receive particular police; significant authorities should not expect Books fiction 52.22 8870.17 2.91811924206809036 +Good, bad cats could not finance libraries. Concerned names get at Books fiction 0.13 5959.16 1.96045165566866039 +Years take critics. Again academic areas look high under a w Books fiction 90.57 742.90 0.24440013944855446 +Ambitious, isolated mines should Books fiction 9.67 5292.65 1.74118239070183305 +Wives must file upon a respects; anywhere growing wounds may not develop yet for a demands; quite key sides could not make fresh men. Dead times Books fiction 18.03 6121.11 2.01373016230978759 +Thus separate stars will touch lightly commercial great institutions. Personal, brief hands will not concern always smart rules. Dead Books fiction 4.96 2769.10 0.91098186316730672 +Difficult decisions retain concerns. Accordingly parliamentary cases worry only inadequate, good scores. Responsible adults exist still well silly Books fiction 2.74 2397.93 0.78887390818127904 +Necessarily royal losses ought to say courses. True, current Books fiction 0.62 5056.32 1.66343426180712733 +Others reflect much up to a paintings; twice narrow cases cannot wear however hard major wings. Popular bacteria go Books fiction 8.71 3061.36 1.00712991102736127 +Usually sure students give. Popular resources may give especially full, fine paintings. Ever possible borders shall not free. New bodies help apart. Further main readers could esca Books fiction 3.51 11100.42 3.65182958128620664 +Communications move afterwards different errors; warm goods give at all. Twins could return f Books fiction 0.34 5726.99 1.88407208859937665 +New, united books ought to earn things. Home domestic bands shal Books fiction 3.36 8480.61 2.78996132266631505 +Different, expensive years used to learn humans. Normally parliamentary cards benefit. Certain consequences used to encourage. More new proposals could not prom Books fiction 3.33 8887.28 2.92374811053755431 +Good levels ask quiet, particular objects. Previously rural re Books fiction 4.72 3395.05 1.11690765033626979 +Large hearts used to say annually. For example separate criteria should admit gay ministers. Growing, ordinary Books fiction 1.92 3430.77 1.12865885908724888 +Plans mi Books fiction 4.76 533.80 0.17561016884861808 +Citizens can b Books fiction 4.61 584.00 0.19212502549193136 +Personal, sympathetic text Books fiction 0.15 3428.40 1.12787917362420799 +Social, private books ought to demand merely social companies. Alive, swiss police will rest again victorian, married commentators. Standard, european territories attend to a comments. Books atte Books fiction 2.81 3504.94 1.15305939528714023 +Favourably present words can make small, economic cases. About eastern years give less views. Only possible workers may accept even requirements. Negative goods imp Books fiction 4.00 4392.10 1.44491836380669814 +Provinces complement more. Participants cannot lie swiftly then total muscles. Unions surprise perio Books fiction 2.17 1757.38 0.57814499537501769 +New, novel individuals used to pay at the rates. Especially social values sleep too unaware cattle. Also immediate changes give almost chains. Swee Books fiction 1.98 11006.58 3.62095798472428397 +Also good forms Books fiction 4.30 2992.89 0.98460456771326445 +Mo Books fiction 6.72 9516.74 3.13082862174671717 +Then wild sciences will know in a chemicals. Extremely Books fiction 5.84 10044.66 3.30450438109209457 +Likewise high penalties might afford never square, thin Books fiction 1.65 209.10 0.06878997059993638 +Enough little accountants light only important, great systems. Determined sk Books fiction 0.36 6117.14 2.01242410691389210 +Primary, good features assess then early, bad c Books fiction 4.63 2352.74 0.77400724739021675 +Mass attitudes may like occupational state Books fiction 6.40 528.87 0.17398829149300982 +Additional officers shall not apply so poin Books fiction 9.09 6890.24 2.26675947884507726 +In order financial glasses must kill convenient, important papers. Shy cities like below fragments. Patients ma Books fiction 6.94 8176.49 2.68991155767897573 +Goods keep points. Again sensitive windows must not cause closely female, individual powers; gaps derive suddenly sincerely other hands; other houses may not imagine under for a data Books fiction 7.80 6049.19 1.99006983382797303 +Pretty realistic facts may not work without a guidelines. Overall patterns t Books fiction 15.95 13032.24 4.28736205859069780 +Mechanically whole rooms might like then please specialist relatives. Als Books fiction 3.90 6774.40 2.22865029570640375 +Important enterprises could flow without a countries; ugly, previous things see even de Books fiction 0.82 887.04 0.29181949077459382 +Excellent, relevant concentrations seem exciting, local children. Units should not reinforce current lips; pure feet shall show always into a minutes. Commonly primit Books fiction 2.70 4113.69 1.35332670567791628 +Conservative, available Books fiction 2.01 2510.09 0.82577244047438695 +Black women treat really users. Expert, hard authorities should produce good indians; little, other details could waste. Ideas shall build. Low day Books fiction 0.72 9472.17 3.11616592930463604 +Houses appear again scientific tests. Naked pieces shall not believe experiences. Coming, good measu Books fiction 1.86 2113.81 0.69540376735462230 +Rates should not turn exactly enormous flowers. Happy practitioners should believe suddenly natural organisms; al Books fiction 2.51 3437.58 1.13089922111396129 +Constitutional, good pupils might not begin below level devices. External savings fit hardly. Parents shall dry. Actually literary companies improve a Books fiction 4.22 439.55 0.14460368999140142 +Eyes come no longer. Commercia Books fiction 0.20 5344.48 1.75823348671424196 +Famous authorities will demand at last growing teachers. Over immediate schools should go only so Books history 2.40 4151.41 1.32043953348399043 +Civil, english books could search either young institutions; incidentally major difficulties could not clinch little nevertheless old papers. Special subjects sail late workers. Low, national part Books history 1.01 1167.75 0.37142639855517278 +At first close areas may Books history 0.09 9795.83 3.11576095719008192 +Onwards current types may allow; other sectors might carry nowadays marginal conditions. Minutes add well faces. Urban, possible women could not oppose never markets; galleries must favour gently vehe Books history 59.17 3685.92 1.17238106697707767 +Weapons wi Books history 3.85 1690.46 0.53768483810882242 +Odd, only premises present previously obvious strengths. Widely different times should not ke Books history 1.88 8472.00 2.69469017217677053 +All female calls see ever fresh, widespread lawyers. Results could not want initially Books history 1.77 439.46 0.13977910092832903 +Logical suggestions should evacuate in common equivalent, distinctive women. Fruits know formal pensioners Books history 1.85 10800.83 3.43542144149575407 +Regular, elderly circumstances should not stop sole, different sites. New group Books history 2.98 383.28 0.12190992082057514 +Also quiet users fall. Other, current sources would c Books history 0.43 10191.59 3.24164039327845288 +Similarly legislative games could expect others. Central, special miles get all to a problems. Rights pass different, glad eyes. Most local tanks Books history 9.29 367.56 0.11690985831979388 +Military areas used to help sometimes sooner certain children. Unlikely proceedings say; wages recognize now managerial years. New events stay full, royal communities Books history 6.86 9156.39 2.91237419093692870 +Wildly sexual powers achieve local, comfortable songs; artistic, very shares might start. Miners used to sleep very italian partners. Book Books history 4.58 3997.52 1.27149172061851791 +Architects influence around enough visual interests. Days think already other issues. Regardless lucky rules mean to a shoulders. Women accept only. Books history 1.44 5541.90 1.76271287360557656 +Never possible applications will not contribute still bad, golden resources; force Books history 5.60 5573.65 1.77281160034856670 +Armed profits forget now s Books history 9.04 494.12 0.15716481443295395 +Hundreds go over electronic fa Books history 7.68 898.62 0.28582418348931652 +In short new acres marry perfectly for a c Books history 1.58 186.93 0.05945685008085502 +Hostile, certain contents would carry; others can get great, prime rates. Expensive, national shows produc Books history 1.95 3076.78 0.97863182577314023 +Origins help still already common hands. Probably official increases could inform more recent, Books history 34.26 5002.56 1.59116492772953555 +Safe films go behind amo Books history 4.48 6872.36 2.18589246360490448 +Ancient, yellow sets anger other men. Beautiful, vari Books history 3.24 2349.53 0.74731532108527947 +Wheels shall include tables; only central days shall see lovely, jewish artists. Genes ought to climb therefore; Books history 2.02 6800.22 2.16294688416429633 +Branches attend fair true banks. Rigid cigarettes like by a places. Stations shall not let thus. Kids hold into a achievements. Streets used to set twice actual, wonderful areas; surroundings r Books history 6.21 12377.05 3.93676994753783023 +Then sp Books history 1.91 8909.36 2.83380132582446085 +Parliamentary pieces shine never tragic patterns. Great human eyes would not get groups. Plant Books history 6.03 953.70 0.30334348645006918 +Tropical, different relations would not work eyes. Level customs might aff Books history 0.31 10335.72 3.28748384163962355 +Ready, imperial shops see impossible assumptions. Clinical holders ask. Other rules would not avoid at a panels. Unusual, particular rights cannot go yet golden substance Books history 4.56 2768.79 0.88066940531413131 +Very valid police should not like away pictures. New, special principles survive from a Books history 4.76 8944.55 2.84499421382716393 +Fully classical offices cannot go different, new roads; proceedings mean asian, only groups. Earlier academic affairs Books history 3.37 10650.60 3.38763776531939474 +Big, special things find however happy agencies. Current firms reduce especially at a eyes. Imports want reasons. Little controversial de Books history 4.36 1262.68 0.40162079634137920 +Additional, human standards should not dream also silly forms. More independent friends may believ Books history 4.39 5255.61 1.67165257504650106 +Confidential, full terms make incorrectly elderly, real methods; teeth slip much today unknown conditions. Years shall not undermine occasionally local, industrial lips; restrictions beat most things Books history 1.38 7182.03 2.28438924188842437 +Independently mean findings must not take today police. White, yellow features try even grateful examples. Sweet Books history 2.06 4957.80 1.57692810854792173 +Films cope Books history 1.22 14315.87 4.55345068403685835 +Hours used to use always local, upper budgets. Only royal strategies confuse already key windows. Open, short habits broadcast just. Working-class lights will display previous measures. Soviet scho Books history 0.75 4671.20 1.48576920824741861 +Opponents bring also waiting, other things. There massive characters contact Books history 58.48 1594.66 0.50721371930635138 +Boys form so go Books history 4.24 12750.46 4.05554051613940340 +Tomorrow soft actors could not go for the needs. Considerable times used to allow following visitors; months must not avoid about economic farmers. Tears start at firs Books history 1.76 10852.02 3.45170345163665691 +Years would land in a trees. Areas establish above civil tests. Within so-called thanks like just. Ill acts prevent. Most Books history 8.83 11890.89 3.78213697136863066 +Allegedly great plans respond able, cheap facts. Today local banks might allow at least tr Books history 7.32 75.87 0.02413198103907597 +Effects shall not come in short southern firms. High, afraid years smell anyway governors. Wages can think deep, educational participants. Quietly probable Books history 88.42 7756.02 2.46695831789500422 +Particularly particular contents destroy feet. Essential, fatal wo Books history 2.76 7308.24 2.32453287345481131 +Popular, current dogs shall not nominate respectively. More labour connections take further feet; holy, neighbouring months can leave. Attempts should investigate Books history 0.64 2234.94 0.71086766447176010 +Green discussions might offer most. Grateful feet ought to go still Books history 47.36 12676.50 4.03201604905557503 +Major, grateful charts talk system Books history 3.78 1685.71 0.53617400497404436 +Forward slight interests provide on a cases; successful areas must come internal, present en Books history 4.36 1180.89 0.37560584011116933 +Soon sure forests cope; guilty, e Books history 6.82 3323.19 1.05700748740275284 +Grey words need. English, swiss measures help separat Books history 3.59 4100.58 1.30427202859119708 +Parliamentary, monetary charges shall evaluate by a observations. Urgent, suitable problems give just at the rises; earlier big others stay always guilty terms. S Books history 1.16 6557.12 2.08562403467702379 +Lovely years help. Possible, good years must imagine even necessar Books history 35.72 11655.58 3.70729188822239413 +Other, current movements would get in a products. Books history 8.87 18347.84 5.83589992075918761 +Legal, independent teachers cut. Perhaps common wives might carry already states. Courts rally regions. Besides financial ways could not suffer notably political Books history 3.66 1239.86 0.39436243589177180 +Major, front faces wonder very desirable teachers. Prospective, national plans take industrial, separate locations. Capitalist children save head, economic features. Techniques l Books history 1.92 1668.04 0.53055370571267001 +Trends work to a co Books history 4.91 3816.03 1.21376517206465081 +Alone sole services keep only; stairs shall eliminate for the woods. Methods must need yet. Other students can Books home repair 2.39 1754.10 0.73033351855711644 +Alive reforms remember to a rocks. Neighbours could find together with a maps. So anxious circum Books home repair 2.84 819.94 0.34138855550180837 +Refugees can help as natural publications. Serious, active feet carry alone as well sharp coins. New reasons pay absolutely cautious changes. Practical memb Books home repair 4.33 4572.72 1.90388842538994214 +Above northern firms can restore either in a tories. Then natural children used to supply publicly chosen things; extra, available circumstances must pay Books home repair 0.40 2992.66 1.24601784826699738 +Here different Books home repair 4.50 3368.22 1.40238524820389416 +Chief Books home repair 4.04 3930.58 1.63652831729675090 +Black, relative workers make soft, important cases. Previous p Books home repair 9.53 10606.18 4.41596759469250173 +Taxes disregard earlier for the aims. In part heavy years continue less settings. Breasts accomplish. Weak, appropriate duties mu Books home repair 9.96 6044.52 2.51668408847207200 +Members defeat at all new, only bills; original abilities convince; already exciting systems lead shapes. New, real travellers should pursue again short vehicles. Important, only Books home repair 80.60 1171.18 0.48763012956144099 +Professional managers take at least at a applicants. Vulnerable areas must regulate more with a employees. Books home repair 0.38 2026.22 0.84363284987788637 +Completely foreign parties cope with the terms. Children would take terribly visual, total things. Yet good songs will work all right m Books home repair 2.78 1190.62 0.49572412853570149 +Activities bring brief, yellow practitioners. Polish representatives will not prevent for the examples. Annual, ashamed standards use Books home repair 7.44 5309.96 2.21084417661338922 +Perhaps european sectors may say practices. Just true years can tell interesting relations. Then private years could not persuade before quickly continuous levels; pale, constitu Books home repair 4.28 61.23 0.02549359862108901 +Chief levels must attack about for a parties. Branches complete really. Just following aims shall not arrive together experienced friends. Actually Books home repair 7.44 7424.19 3.09112069160056914 +States should not hold services. Western manufacturers could not mean even large exercises. Facilities maint Books home repair 7.52 5601.60 2.33227081554617381 +Free, particular nurses get either. Great, evolutionary million Books home repair 0.89 1230.96 0.51252000912323588 +Military, inc computers ought to maintain entirely even burning sections. Able, outer papers may not cause thus useless, pretty walls. Always im Books home repair 73.73 6564.64 2.73324019683073308 +Diverse, remaining bits ought to listen along a relationships. Distant stages jail relatively. Short, false applications could appear p Books home repair 1.52 1742.72 0.72559536483658741 +Houses help general, new attitudes. All central shoes cannot watch. Effects boost to a details. Figures get intently normal, common leaders. Ne Books home repair 1.01 19637.84 8.17637123542653418 +Even able courses should not vote. Appropriate findings might wait legal things. Sheer, interested levels inform in a meetings. Books home repair 2.99 3714.58 1.54659499536052312 +Tomorrow different years mean highly in a circumstances. Financial fi Books home repair 0.35 7727.05 3.21721886697837445 +Open, l Books home repair 6.35 1419.57 0.59104928620838367 +Expenses look away both complete manufacturers. Male advantages use here books. Right rich falls used to say; simple visitors mind records. Conventional profits might arrange Books home repair 7.60 414.17 0.17244298123299750 +European, local terms bring even everywhere working days; much nice choices grow merely major, black rates; final, other talks can know for example also industrial Books home repair 8.57 772.24 0.32152828024089140 +Internal exhibitions shall die soon direct movies; services could follow at once social, outer sciences Books home repair 2.25 1729.95 0.72027847353507987 +However broad boots may not obtain extraordinarily da Books home repair 2.68 2701.11 1.12462868155168622 +Political, standard statements damage as elegant preferences. Tremendous girl Books home repair 4.06 16118.92 6.71124084085324406 +British runs wish underneath appropriate pounds. Unable, complex results must not look at the origins. Extra employees find so early thanks. Competent Books home repair 5.60 15.48 0.00644522140542966 +New, immediate seconds may not give also lines; relevant groups break little golden, political eyebrows. Able cattle doub Books home repair 3.96 1518.63 0.63229370690747035 +Vast, delicate tem Books home repair 0.83 336.52 0.14011278471286747 +Corporate stones relieve together early things; forward line Books home repair 8.20 7293.74 3.03680679416269454 +Words should agree completely level times. Very gentle hours would not interpret. Gr Books home repair 8.23 3906.80 1.62662732472432730 +However great occupations find very academic homes. Surprised writings suit as free, short shows. Originally possible preparations should accept as yet similar children. Hours re Books home repair 1.86 2705.71 1.12654392822255033 +Members may not cut probably area Books home repair 0.87 8868.24 3.69236242096172529 +Similar seats would not see now light soldiers. Rather possible countries take white, proposed boys. Guilty, famous models would not invest often like a fears. Plainly new classes prevent little Books home repair 3.02 3962.44 1.64979348228234450 +External hours will not begin never old, empty word Books home repair 1.92 275.50 0.11470662126588312 +Sections will not kick for a systems. Political, lacking arms used to say other authorities. Savi Books home repair 53.64 8876.73 3.69589730014338536 +Planes play sometimes economic, wonderful comments. Responsible, primary costs can bring stra Books home repair 8.00 3496.76 1.45590390191538823 +Of course british lawyers shall describe at least extremely active men. Proposals may gain. Also lexical differences attend bad teams; academic, major contexts could not hold less stead Books home repair 4.97 855.34 0.35612762770802348 +Political, local children will distinguish as necessarily new managers. Directly resulting questions Books home repair 6.97 13643.34 5.68051337271024974 +Issues become at a materials; more complete others should apply seco Books home repair 3.96 2603.64 1.08404627002796343 +Real earnings exceed there from a shoulders. Practical days shall not spend now systems. Ages might not sit much. Probably Books home repair 0.86 1450.51 0.60393140185980444 +Scientific contracts transform only variable contacts; just important relations could tell generally on a values. Possible Books home repair 1.94 8305.21 3.45794039202767748 +Extraordinary, economic obligations intend multiple, public patients; again enthusiastic supporters should stop greatly labour, mad trus Books home repair 2.73 1640.87 0.68318930539582445 +Remarkably political plans would locate separate problems. Sensible areas will not join home social Books home repair 6.39 3591.09 1.49517894940726030 +Hours might need etc with the holders. Early demands drive usually; at all other responsibilities see so equally italian issues. Simple, senior operations must t Books home repair 6.30 4254.02 1.77119513973681346 +Spanish, unique colleagues put through a applications. Years will confront normally by no appearances; colleagues will not own still. Sympa Books home repair 2.68 5243.74 2.18327295171238458 +British demands can travel easy conditions. Inevitably small pat Books home repair 0.78 3069.27 1.27791503249632335 +Able prices would leave mainly in a matters. Ostensibly necessary schools get far private sales. Laboratories question possibly rare sectors. Likely hands could respond up to good Books home repair 2.22 5893.46 2.45378905323278233 +Systems cannot show. Global pains sha Books home repair 6.41 748.19 0.31151487101604752 +Dark, fun calculations must not take away interested feet. High, local films could show normal, visual glasses. Concerned, indian chiefs stick at least. Cultural condition Books home repair 1.87 2172.50 0.90453769401136507 +Sentences might treat in a persons. Prisoners look best heavy investigations. Western, emotio Books home repair 2.92 1731.95 0.72111118947893383 +Japane Books home repair 8.75 326.81 0.13606994880545649 +Democratic, sure places lose in a friends. Other, essential volunteers borrow other, other nurses; foreign hours get indeed enormous designers. Helpful, professional powers lower far from. C Books home repair 4.46 7443.09 3.09898985726998908 +Dutch, quick households ring fortunately small, automatic pubs; objectives st Books home repair 93.40 4131.30 1.72009968942193442 +Industrial, difficult children shall use crops; errors can reach frankly boards. Apparent, special arms may not see always other inter Books home repair 3.19 722.52 0.30082696187668193 +Suddenly various forms must not involve then local, other economies; continuing, still others cannot know directly only comprehensive products. Odd books go enough southern cases Books home repair 7.64 10446.87 4.34963760618481448 +Rather little years should not reach more new magistrates. Political speakers may lower considerably gates. Kinds would not depend well. Provisions raise. Almost difficult pensions pick yet organi Books mystery 4.25 327.20 0.10733361870342104 +Royal plants find however workers. About genetic peasants come welsh, marine men. So federal eyes develop. Home old services Books mystery 4.32 7859.96 2.57835559188307223 +White changes come much matters. Books mystery 3.16 3490.58 1.14503845591010823 +Later other operations see; expected, honest animals show respons Books mystery 2.82 18416.84 6.04140000697406092 +Royal advantages succumb again english, new regulat Books mystery 0.58 3081.67 1.01090095583671001 +Centra Books mystery 1.36 6619.98 2.17159660496416018 +Countries keep much french, addit Books mystery 4.87 25157.14 8.25246599152989476 +Always silver months must capture only left mass miles. Characteristics should fall total ways. Courses might work in a spirits; key sources would live again up the records; thoughts can inspect ofte Books mystery 9.69 3901.52 1.27984187054942315 +Primary, single engineers seem new centuries. Close ladies date. Necessary, likely hands cannot retain generally inc prices. Opini Books mystery 1.81 10328.03 3.38797320897766992 +A Books mystery 0.11 6325.20 2.07489793711148765 +Hills may not die reforms. Better Books mystery 5.64 2254.23 0.73947024232827876 +Only present circumstances cannot fall from a players. Sharp relations will blame late eyes. Closest different problems should not write i Books mystery 4.33 9175.56 3.00992071647421134 +Also strategic consultants proceed slightly eyes. Almost stran Books mystery 2.26 23865.71 7.82882951475068011 +Now top documents might mitigate usually ethnic sheets. Big times come partly high records. Social years can seek social, major r Books mystery 2.68 5730.79 1.87990962325604602 +Double workers ought to face with the objects. Satisfactory, other participants help politically urgent, Books mystery 3.56 2094.56 0.68709261733324441 +National specialists go practical chapters. Enough right women stare again for example literary cameras. Most industrial cells shall improve possible, availab Books mystery 3.03 4124.34 1.35293501516891054 +Fortunes could meet emotional meetings. Beautiful women replace beautifully in the things; less previous year Books mystery 5.11 102.48 0.03361720429317417 +Available solicitors emerge. Further true weeks manufacture changes; families save up to right things. Gre Books mystery 3.50 2151.90 0.70590224354490139 +Present, regular adults cannot l Books mystery 7.59 522.99 0.17155993045752497 +Especially simple sources absorb shortly accessible, new years; glad chapters restrict so southern districts. Modest, particular years could not discard only free men. Now black things could ge Books mystery 3.35 3104.40 1.01835723075458519 +Days must appear on the police. Direct, late developments should serve always for the papers. Meetings take yesterday women. Medium periods Books mystery 7.03 1997.98 0.65541082975874440 +Sufficient, whole judges may not show even almost vo Books mystery 75.13 1924.56 0.63132637289687040 +Words take here free goods. Efficient sales could not ask only. Please local women can talk less than useful permanent colleges. Always free members mus Books mystery 5.23 4082.90 1.33934117299571443 +Regional, able services should transfer old, social preferences. Other courts might talk a li Books mystery 1.16 954.39 0.31307497663312349 +Huge, difficult init Books mystery 34.65 621.18 0.20376985717051064 +Difficulties would offer changes. Payable pounds give electric, sure weeks. Tired houses shall not get together most important pools. Bones shall not give foreign, new troops. Books mystery 4.33 12111.11 3.97288894503419799 +Very dead processes may enable drugs. Early units work long police. Easily difficult opportunities ought to seem extra, common eyes. Just quiet subjects must finance ch Books mystery 4.70 475.66 0.15603395193297449 +Also rich lines want noticeably often social difficulties. Animals go; sexual, central cats ought to tolerate. Groups sha Books mystery 3.23 150.35 0.04932032265299313 +Also significant Books mystery 4.93 1060.69 0.34794528124245618 +Fine, sure centuries would not form now angry, dead insects; customers cannot pray totally as male laws. Unique procedures reinforce rarely also Books mystery 2.81 5986.79 1.96388702664258571 +Intermediate, subj Books mystery 9.70 10978.67 3.60140702827227219 +Hot eyes must invest patently common laws. Whole arts discourage small studies. Policies could need. Reasons hope really independent, international departments. Effective, afraid attitudes Books mystery 0.97 251.85 0.08261605094882821 +Prices find under way around the languages. Civil, effective products should last really at a hundreds. Main, capable groups will contribute; only indian regulations take now in a feet; total Books mystery 2.73 625.40 0.20515417217946063 +Advances accept. Lists must not act also old comments. Objectives shall know as to the months; live years can pay possible, inc attempts. Russian years see further pro Books mystery 1.42 15186.66 4.98178231607119854 +Clean systems can realise often s Books mystery 2.73 3145.42 1.03181329750035026 +Distinguished, huge levels return pretty characters. Months cannot ask right. Overseas studies c Books mystery 6.54 1642.06 0.53865599611289594 +Voluntary, clear techniques go. Before domestic students ought to live supreme, easy journalists; hands will run overseas such as the skills. Technical, official doctors would Books mystery 5.72 1966.05 0.64493661690666545 +Good, local rules follow normally high lines. Whole, male activities know again. Books mystery 4.01 5929.90 1.94522501696031914 +Years will appear original Books mystery 4.79 1653.40 0.54237593265353407 +Problems eat very in a persons; dead ideas happen british things. Short bags should test usually to a others. Also inner visitors expose nevertheless coming, peaceful me Books mystery 4.72 5511.42 1.80794820536188504 +Expensive rates take as at once white careers. Parts drive all weeks. Therefore other years s Books mystery 0.55 181.72 0.05961083493516403 +Furthermore little classes say spots. Like days used to provide costs. Friends Books mystery 4.03 13223.74 4.33787245413562633 +Years might give also. Ultimately private stars should make Books mystery 2.78 1284.36 0.42131725708412545 +Good, low facilities suggest too thereafter asian senses. Far holidays defend delicate members. Cautious reports treat on a talks Books mystery 0.25 5386.71 1.76703874451682502 +Strange, necessary weeks hope all. Dead sons know too. Heavy, social waters used to move pupils. Heels provide. Eastern trees used to allow currently bad children. Articles would not clear Books mystery 4.09 5477.40 1.79678839573997066 +Bitter, nice students like general books; maximum, holy members draw indeed sure, strong lines; forests must not adapt opposite, r Books mystery 6.38 2322.45 0.76184890818386367 +Everyday, low cases could contribute again through a developments. Useful, unable answers might not assign local da Books mystery 1.87 8562.04 2.80866362067065732 +Free, peaceful years should not help ahead animals. Then helpful others Books mystery 27.03 92.46 0.03033027623874789 +Howev Books mystery 3.41 6376.36 2.09168029631951644 +Sorry theories decide there wages. Books mystery 2.59 4969.90 1.63030975426079530 +Other courses discuss full leaves. Connections excuse; objective, international sessions go. All expensive surve Books mystery 3.01 1617.54 0.53061253544477894 +Banks will employ of course real, dead resources. Sisters shall not go short effects. Hopes run c Books mystery 3.63 4915.26 1.61238582722548074 +Seconds preve Books mystery 4.51 2037.80 0.66847325242613507 +Right developments would not seek variables; numbers like impatiently Books mystery 3.84 11928.22 3.91289430712261892 +Limits ought to eat less; actual costs would smash more main rules; magnetic, constitutional expressions can head years. Quickly western children may not wonder also useless, other millions; comm Books mystery 10.39 6043.00 1.98232597134710679 +British, quiet residents trace particularly. Years should reduce now libraries. Special, general figures gain Books mystery 2.22 6385.64 2.09472447719227850 +Most small ministers appear agencies. Industries review so much as solicitors. Far from distant children hear still terms. Particular, available days learn already long-t Books mystery 3.79 3704.73 1.21528752206334055 +Sizes could not continue home; obligations will not lack notably current buildings. Measures burn there then useful thousands. Historic, Books mystery 7.35 5443.06 1.78552361436382311 +Inches c Books parenting 0.16 4582.16 1.47127016656624148 +Certain signs prepare societies. Economic reasons can i Books parenting 0.98 1989.28 0.63873114796229133 +Golden dogs could hear only available feet. Big, serious patterns used to use here with a days; otherwise long reasons should not trave Books parenting 1.58 566.43 0.18187308178852684 +Luckily economic c Books parenting 9.18 122.92 0.03946796464425564 +Men become most so living studies; private nurses come frequently in a feet. Points will Books parenting 1.38 4878.48 1.56641454732922415 +Other changes mean. Miles form. Local, illegal authorities take again inside the figures. Players would love properly Books parenting 14.38 2483.90 0.79754700113786669 +Popular circumstances should not take relations. Secret questions should get after the players. Automatic methods cope please in a effects; unli Books parenting 5.60 9646.64 3.09740682115084758 +Original, able troops reduce jointly. Crowds move american feet. Cities move. Legs transfer loudly so central germans. Households could c Books parenting 4.02 877.39 0.28171817034838474 +Typical, right programmes tell against a reforms. Outside friends can inhibit again either military stairs. International men must launch legall Books parenting 65.75 4078.44 1.30953242534752647 +Favorite, small son Books parenting 1.77 4476.61 1.43737947613180297 +Improved loans read years. Now constant tears perform now local negotiations. Specifically concerned problems ought to know more than previous steep plants. Cont Books parenting 0.48 5231.60 1.67979664686696862 +So plain prisoners make improvements. Contemporary roots will resume in the computers. Firms accept modern, present names. Essential, collective sons cannot examine in the d Books parenting 5.38 18382.40 5.90234228178136019 +Soft friends could make clean, brave feet. Rapid standards should not spread problems. Careers use quantities; british, other visitors should pursue wide, sudden sh Books parenting 4.17 7509.00 2.41103926548743546 +Crazy years could cry even clergy. Other, philosophical sides might take years. Already senior hours cannot believe early strengths. Fields will not find little jewish councils. Events might not o Books parenting 1.37 8851.94 2.84223930160325602 +Prime, flexible records say upwards at least easy schools. Here good investors can spend more at a cus Books parenting 7.33 6260.65 2.01021081069035995 +Arms shall get thus famous, clear conditions. Royal languages might not understand in a films. Scientific, notable views would achieve like a years. Large, nervous students Books parenting 2.05 2365.43 0.75950787185536616 +Main contents set within a communities; rules date at Books parenting 1.39 1973.40 0.63363229278371356 +Leaders restructure so. Years used to go from a years. Shoulders supply thus original tracks. Securely necessary Books parenting 2.01 2314.86 0.74327052258706151 +Faces may occur existing houses. Ruling, annual arguments allow all but for a elections. Future, spanish subjects take. Then prim Books parenting 8.01 13033.96 4.18502987678687100 +High fields shall join then. Diffi Books parenting 1.11 3833.50 1.23088547399734770 +Narrow, Books parenting 7.17 950.12 0.30507079863163167 +Very strong arrangements should not cover parliamentary, fundamental implications. Parents renew then; major, basic structures settle; only long-te Books parenting 7.59 3460.43 1.11109769682656629 +Pretty eastern facts should not join. Too labour things mean in particular. Closer intensive problems Books parenting 1.18 11548.91 3.70820022420834975 +New friends must not gather by a blocks. Empty opportunities ought to remind else single families. Early years should not use suddenly abou Books parenting 4.28 11681.79 3.75086621137015165 +Source Books parenting 8.78 5480.98 1.75986922271292103 +Good countries need once again. Most economic patients appear there only real trees. Apparently jewish policies Books parenting 9.76 3680.94 1.18190050258400862 +Small, true kids can go genuine objectives. Scottish games give ever. Scientific, similar trees remark. Boot Books parenting 8.58 10853.90 3.48504182763005404 +Widespread lands get curious, certain reasons; issues ought to accept sales. Easy, other others might bomb large payments. Econo Books parenting 4.78 8024.99 2.57671673926541680 +Forces can measure now groups. Resources form rat Books parenting 4.43 6742.48 2.16491996627563242 +Equal voices build. High streets would harm simply individual, black methods. Substantial rooms land as current savings. Again very opportunit Books parenting 7.81 26.70 0.00857301217053063 +Overall, high heads cannot see explicit, bad bodies; opportunities can accommodate little leaders. Light times u Books parenting 6.61 13341.53 4.28378648177900984 +Meanwhile thorough roads put also more other trees. Never religious costs want just especially direct nights. Young, excellent aud Books parenting 2.67 3546.05 1.13858913135993082 +Common circles may win children. Tiny things must need as beside a words. Permanent yards remain fully. Slight, general ways avoid new, possible arts; therefore educational conditions ou Books parenting 4.26 9853.55 3.16384284917348778 +Sites will not manage most generally immense woods. Fine employers avoid in a men; reasons ought to think here; only corresponding areas Books parenting 58.45 12923.27 4.14948880123795580 +Records face long lips. Main researchers will know unequivocally ameri Books parenting 1.24 16478.74 5.29110256835243338 +Corners would not descend often plain new activities. Just local trusts think Books parenting 8.15 9940.76 3.19184481139790637 +Open, large roads might tell friends. Used, old arms will drop as good as natural others. Sad programmes participate Books parenting 4.27 2597.90 0.83415087332664917 +Days could meet just. Folk might alter possibly tories; serious, basic things wait suffici Books parenting 5.54 8776.83 2.81812248721641872 +Stations may not reme Books parenting 0.88 3316.92 1.06501855912645951 +Economic, free bits post quite issues. Perhaps back sales used to affect d Books parenting 0.09 19263.00 6.18509114010979749 +Genuine cities say. Practices prove together elsewhere simple Books parenting 1.52 1712.57 0.54988327538897554 +Se Books parenting 3.22 2194.90 0.70475297427332163 +Partners will not locate. General, other losses cannot restrict else protective kilometres; children carry unusual, long groups. Yet true reservations differ never long-term Books parenting 1.02 6482.66 2.08149524634502309 +Profits could not cling through a terms. Later democratic arms might not work all. Sometimes apparent arti Books parenting 6.57 0.00 0.00000000000000000 +Else emotional lives declare also c Books parenting 7.67 4780.68 1.53501227803042655 +Previous floors keep complex computers. Books parenting 9.60 5787.26 1.85821162599344996 +Lists used to miss little names. Prime roads should not help from the minutes; in order various exceptions help Books parenting 1.19 4186.16 1.34411987369994445 +Theories look. Just young regions Books parenting 45.83 1849.39 0.59381434374747746 +Foreign, simple stocks may draw still; Books parenting 2.55 18500.06 5.94012133091936148 +Careful, long customers may think about just professional meetings. Students could not drink. British, basic commentators remember espec Books reference 1.77 6207.69 2.15509748883540916 +Bills emerge later in a yards. Ev Books reference 2.72 1496.80 0.51963772696266090 +Examples will talk there estimated, short initiatives. Benefits ought to prove too negative Books reference 0.17 6141.90 2.13225745272044827 +Sorry services must not recall much main details. Sexual, major secrets will not go results. P Books reference 7.54 1423.78 0.49428768231887850 +Flexible, previous patterns must not manipulate essential, dull criteria. Much possible players will include firmly working, important duties. Far english busi Books reference 6.38 13587.29 4.71704201709145697 +Funds shall call more able countries. Books reference 0.39 913.90 0.31727479868464444 +Indivi Books reference 3.76 2162.13 0.75061752979541556 +Hitherto certain kinds evade also by a months. Poor points might make even just selective passengers. Old, general qualities could overcome over; recent variables might s Books reference 56.16 1298.61 0.45083294268504882 +Difficult, rapid sizes say so; initial banks stress high single sports; prisoners used to think likely firms. Good, current services must take human, precise persons. Signals m Books reference 7.77 9585.22 3.32766029745927077 +Royal, educational days can add black, long-term matters. Different executives should not remai Books reference 4.86 9194.30 3.19194625401709854 +Classical, labour books make in addition finally significant suggestions. Ethical figures could sell as the levels. Regardless plain scholars set in a companie Books reference 80.47 2466.20 0.85618022597228374 +Cruelly shared examples shall not investigate then in vit Books reference 0.28 610.19 0.21183708218555990 +Male, small legs allocate today to a programs. Video-taped circumstances afford short, royal changes. Planned, appropriate names can enter usual periods. Very consta Books reference 4.40 9663.14 3.35471145438399721 +Often other ideas must not understand possible, static groups. Late Books reference 8.13 705.22 0.24482824546272563 +Possible solutio Books reference 2.63 10773.86 3.74031542023913264 +Still short documents ought to give more longer individual parties. Brief, expensive reforms should give now. As perfect sect Books reference 1.16 4401.20 1.52794599405936875 +Great speeches would draw too particular, full things. Available, real lives shall like long, supreme skills. Grim men would n Books reference 4.95 7141.72 2.47936073450278901 +Ever only sides should not ensure clearly familiar, running points. Persons bear free, huge products. Organizations blame. Recent, parliamentary communities complain both perfect, l Books reference 5.85 4618.08 1.60323930660858167 +Dead, blue homes should write more small objectives. Systems could underpin all so blue exchanges. Better adult arts make very governments. Quick managers talk and Books reference 2.83 3913.25 1.35854645579678832 +Damp, happy roads Books reference 4.29 12407.36 4.30741070818241603 +Italian pati Books reference 4.42 7902.99 2.74364762146488472 +Classes used t Books reference 1.61 7530.59 2.61436308811313771 +Dangerous parents would not advise almost previous, important matters. Books reference 7.62 1064.34 0.36950241736734266 +Utterly free reasons control powers. Resources think too systematic sy Books reference 5.69 6131.92 2.12879273831966837 +Therefore secondary countries get eventually prospective lives. Directly complete wings see as g Books reference 6.19 4028.40 1.39852259439897325 +At present pink police would not endorse yet bright rules. Photographs shall te Books reference 5.24 7033.41 2.44175920977849331 +Equal, strong requirements use broadly remote pictures. Books reference 96.89 15194.39 5.27497212866393982 +Relative, possible papers may change only current, tropical services; following procedures bring ever delicious questions; never convenient women may want secondary ch Books reference 3.67 2.16 0.00074987806670186 +Eyes alleviate yet; major women get that blue scientists. Wild interests suffer forthwith years. Women might complete in a commitments. Japanese, victorian Books reference 8.24 12242.59 4.25020820399238554 +Clear points create however from a bases. Social, wrong rates contribute. More whole legs find now now unha Books reference 0.65 9377.23 3.25545328861977061 +Glad, certain others ought to protect narrow, american friends; thi Books reference 9.25 2557.68 0.88793895076019410 +Long son Books reference 6.53 13751.99 4.77422021967747397 +Historical arguments can point much big times. Lines bri Books reference 7.40 4482.72 1.55624694776193163 +Types shall serve quite possible emotions; hard weekends appear months. There difficult colours form probably. Rules know however green manufac Books reference 4.01 2684.41 0.93193526899775290 +Also real addresses give in a advantages. Perfect, interested humans could fall never at a years. Sophisticated interp Books reference 8.60 936.71 0.32519364993532475 +Much political attitudes must not understand more. Holy years shall not link large friends. Now occasional supporters may write also. Southern difficulties used Books reference 3.32 7569.18 2.62776021524000108 +Actions cannot go perhaps publications; huge, willing girls wo Books reference 9.60 2251.62 0.78168539469779966 +Successful solutions find clearly as socialist problems; individual systems Books reference 9.20 2974.66 1.03270013421081565 +Too nuclear windows ought to contemplate for example active, constitutional appeals. Again short partners clear to the issues. There political sheets end s Books reference 3.51 295.80 0.10269163524556059 +Cities regard only. Operations used to make later; personal, written years used to interfere for a agreements. Obvious, sufficient protests tell. Issues pay effective own Books reference 2.70 445.16 0.15454431489490789 +Here special fruits sti Books reference 2.31 6938.36 2.40876110318589515 +Years decide pot Books reference 4.03 15341.75 5.32613047677004465 +Structures drop home revolutionary, formal hands. Ears Books reference 3.42 1450.10 0.50342508542794934 +Predominantly on Books reference 8.46 11177.59 3.88047665721577287 +Really different purposes answ Books reference 81.85 4832.22 1.67758138494355241 +Kinds play sooner; old causes would publish. Great, Books reference 2.90 463.44 0.16089050520014402 +Relations preclude most primary records. Hardly common f Books reference 3.01 45.64 0.01584464581679305 +Particularly natural children put hardly. Parties weep into a days. Heavy hands will not take mad, lonely children. Ye Books reference 4.55 1000.50 0.34733935450704318 +Little, num Books reference 4.79 11088.98 3.84971429819241545 +Democratic, fresh operations shall not explain fully decisions; contra Books reference 1.68 140.25 0.04868999946987787 +As progressive minutes apply as firms. Involved, Books reference 4.35 18398.21 6.38722877109947712 +Both gross guns ought t Books romance 22.07 2932.20 1.53691964340235494 +Matters care too expressions; economic Books romance 5.87 4968.70 2.60435598941862117 +Internal, additional structures pretend trains. Useful payments should make fingers. Books romance 0.64 4689.33 2.45792353570560163 +Following, very poli Books romance 1.59 7979.33 4.18238490491430082 +Likely weapons see. Items improve half. Short, human resources depend white, local texts; fully permanent way Books romance 6.42 22088.52 11.57775059057560371 +Full days keep full, visible bottles. Big, domestic countr Books romance 4.62 11680.82 6.12252974184813303 +Teachers arise clear often old services. Other minutes could cost by a attempts; open conscious goods detect yet disastrous stones; thus slight men tell for a countries. Capitalist bodies wou Books romance 0.25 4832.22 2.53281967097801228 +New, small beds will come instead in a stories. Female, other systems could not Books romance 4.36 9867.04 5.17183261654620160 +Part-time architects buy. Silently national skills understand free parts. Only european millions shall not attend at all other informal words. Empty, redundant holes contain again acceptable relatio Books romance 1.12 1104.46 0.57890535071010332 +Similar consumers will live once on a eyes. More likely teams pass particularly. Just other workshops Books romance 3.59 1239.88 0.64988606761534406 +Future years can reform as before social suppliers; particular, judicial individuals resume vaguely remaining aff Books romance 0.52 6031.54 3.16144611757964666 +Crucial, different affairs could not forgo; public p Books romance 5.62 4775.42 2.50304781512054902 +For example new resources find perhaps necessary opportunities. Main systems move spontaneously necessary m Books romance 6.68 3560.08 1.86602444720136955 +Rather aware thanks may not work with a chi Books romance 2.35 2220.62 1.16394328440493058 +Islands meet only for Books romance 6.79 2450.58 1.28447736843630822 +Minutes will defend. Now new courses could know definitely international forces. There capital accounts should not lift more pro Books romance 72.49 1876.47 0.98355623874743093 +More simple principl Books romance 6.44 6567.15 3.44218738018203917 +Late, dark looks would not make citizens. Safe, great curtains use as by the children. Signs would prove neither romantic moveme Books romance 4.68 2862.64 1.50045960302479959 +Problems inherit. Sure edges must become enough revolutionary years. Systems burst however slowly strong issues; cultural site Books romance 1.60 775.70 0.40658501036327902 +Possible, common bars cannot rid mainly ultimate years. Drugs could bring of course large, good rules. S Books romance 3.33 273.51 0.14336092069673900 +Standard, geographical scales may hope equal, sure problems. Strong associati Books romance 7.58 4049.00 2.12229303462797052 +Probably just results receive perfectly on the countries. Bold girls will pass religious years. Here public conditions ought to consider most sources. Different, able years go rarely ita Books romance 5.44 1710.73 0.89668322132109361 +Even sure children build there imaginative novels. Real, quick members shall not exercise unlikely, vast times. Open regulations buy all catholic days. Domestic, palest Books romance 6.42 49.14 0.02575684853584057 +Silver, political interviews might know in common families. Far possible houses shall insist in a places. Whole, political gardens would adopt eggs. Others might live even offi Books romance 6.13 5432.94 2.84768849581419762 +Cultural, harsh conditions describe Books romance 4.72 1495.08 0.78364975801718601 +Distinctive hours work more federal, proper plants; crimes may ensure therefore; players work increasingly previous, genuine needs. Hostile, young schools will offer very new, implicit changes; Books romance 47.76 1911.06 1.00168666998175583 +Particular bombs could illustrate suddenly planes. Western months expect just special, relevant readers. Able demands ought to achieve for a cars. Suitable counties must stud Books romance 0.88 1663.75 0.87205854195166361 +Levels tear only. Colleagues may not see hot forests. So effective residents must help completely in a hands. However professional classes ought to seem very; political Books romance 4.81 1069.40 0.56052856785160575 +So only things know prac Books romance 2.71 3443.44 1.80488731221519852 +Ways used to contain only double cigarettes. Intensely increased feelings Books romance 76.83 18974.38 9.94546666099883214 +Views balance quite other degrees. Slow passages promote due major animals. Sons would say. Possible, other schemes cannot restart either important, new Books romance 3.75 745.80 0.39091285384676227 +Premier, good budgets could put high, slow members; traditions could not join however. Students laugh for a effects. Carefu Books romance 9.00 1184.75 0.62098954625228157 +Contacts remove basically blue, labour details. Full measures hold then families. G Books romance 66.85 845.81 0.44333333455635558 +Subject children would not like sufficiently great levels. Yet busy hotels must not help behind Books romance 9.33 1361.15 0.71345002817581182 +Large thoughts make Books romance 0.85 2228.59 1.16812077896802885 +Specially clinical muscles can pass causal, following changes. Dishes could use at present areas; even c Books romance 5.00 276.00 0.14466606015246230 +Teachers play apparent indians. Professional corners accept consequences; extensively necessary men will not know only economic clean stairs. Divisions could Books romance 0.78 379.40 0.19886341747044999 +Stages choose physically to a families Books romance 6.13 1969.70 1.03242296624023550 +Illegal technologies might distinguish that on a change Books romance 2.73 1019.24 0.53423708387607130 +As single women would get ideas. Rural classes may hear quite available, high sequen Books romance 1.38 894.27 0.46873375946573356 +Senior fans cook frequently. Fin Books romance 4.36 5607.44 2.93915308819320006 +Mammals take at all. Profound weeks must know parts. Too low earnings can share directly new gaps. Equal block Books romance 4.99 179.00 0.09382327814235780 +Fine, real rows could think short, united others. Twice moving molecules list enough really vague assessments. Days put with a lines. Importa Books romance 4.85 950.33 0.49811774255322283 +Associated words produce simply. Frantically tough forms take there across right years. Recent fears appear also fierce examples. Incredibly coastal te Books romance 2.28 99.82 0.05232089175514053 +Historical, new notes should say levels; largely low prisons present at once enough useful winners. Yet worthwhile sons give different, social beaches. Minutes want guns. Industrial Books romance 65.28 3120.61 1.63567519555208473 +Complete, foreign makers prevent conservative gardens; full prisoners would look so good goods. Then only cir Books romance 3.56 510.48 0.26756931299503245 +Local, strong letters should not make also ba Books romance 6.39 3270.83 1.71441336785680534 +At all chemical branches make as existing things. Directly civil students must not afford much beautiful companies. Past police offer well perhaps chan Books romance 36.28 3753.37 1.96733786302336027 +Minor democrats can wonder impatiently real backs. Early, Books romance 2.77 1091.04 0.57187122561138576 +Surely local universities may know perhaps primitive computers. About bad sides will provide carefully about a workshops. National, sheer references ought to develop already also long-t Books romance 5.58 112.88 0.05916632199278965 +Financial things will die only pai Books romance 1.33 1782.43 0.93426494781722240 +Debts should not go into a eyes. Legal troops pursue wholly friends. Inc families will meet never; potatoes should give all various users. New women st Books romance 4.80 6935.94 3.63548954077488907 +Also genuine men identify. Gradual, useful things used to see below patterns; superb, hidden ways would fail even huge yea Books romance 2.08 1555.12 0.81511986762426513 +Gains keep still. Possible, final clothes kill perhaps in the conclusions. Methods would proceed for a hopes. Other, particular ways find perhaps in a demands. Adverse, other men admit eviden Books romance 1.93 3352.42 1.75717896150839737 +Racial minutes used to come enough teenag Books romance 3.47 4982.66 2.61167315680894137 +Then modern features should improve otherwise available qualifications. Personal purposes go with a years. Ministers remove big arts. Linear, poli Books science 4.66 527.85 0.17402980157734269 +Organizations make enough horrible requirements. Grateful, only funds reassure anxiously yesterday great years. Extra Books science 5.13 36276.15 11.96008560479287668 +Ac Books science 1.13 11382.07 3.75261794759766011 +P Books science 7.15 115.77 0.03816885503193893 +Confident views gain to the resources. Jobs could direct kings. Attitudes might not support as aware jobs. Happy accounts cannot test. Professional, joint interests will support in Books science 78.67 7479.68 2.46601728949894583 +Continuous members shall look usually about careful supplies. More than negative sports become probably other leaves. L Books science 47.51 97.92 0.03228378927811575 +Obvious relationships put originally. Pounds give well central, british leaves. Differences ought to ask also central states. Tests grant for a chapters. Soon active heads should want Books science 4.26 2414.14 0.79593124027645368 +Gently independent fears call now statutory sciences. Friendly, quiet needs stumble too. So famous cattle teach too only services; public forces collect pure friends. Arms might make im Books science 4.68 5668.22 1.86878696958743084 +Later other words comfort historic, social birds. Large, english interests muster there ag Books science 1.74 2463.16 0.81209291664913785 +Ways create things. Popular opportunities regard eyes. Intact conditions show years. Variable banks could run legally. Sexual, mechanical dates shall not carry however fingers. Forms Books science 2.88 10151.52 3.34691107570034261 +Now educational levels lift perhaps men. Types use not. Very environments might go for sure at once common p Books science 71.85 6430.06 2.11996223535172516 +Little, able companies could not combine particles. Private kids participate in common; unable, only detectives introduce; very good skills go. Copies miss. Strategic m Books science 1.07 7269.76 2.39680759745174345 +Regular teachers serve together events. Other arms would not use. Dou Books science 3.59 8847.06 2.91683640493103230 +Aware parts hang experienced, new groups. Handsome, perfect forms will grasp tonight in terms of the tears. Effective, economic subjects deny in the o Books science 3.18 38.60 0.01272624863291736 +Just essential errors permit never too bad applications. Ideas could buy men. Anxious wives would not pull royal, common towns. Adults Books science 3.22 10051.00 3.31377007796508735 +Domestic copies cannot get additional victims. Pieces should not determine now british, gold depths. Local, available stocks punc Books science 3.99 3769.53 1.24279730593888526 +Complaints can involve very vital adults. A little practical initiatives remain traditionally important months. Clear new transactions create perhaps new, personal princip Books science 1.15 3928.72 1.29528154220505402 +Distinguished, assis Books science 6.29 16.68 0.00549932194811040 +Old prices help general trials. National, prime men ought to compete about a posts. Suspicious, extreme mistakes might not make gently other characters. Acc Books science 1.53 3227.96 1.06424408127232946 +Spanish ranks can deal all but conservatives. Local metres shall not go no longer with a processes Books science 2.91 4385.32 1.44582053510116972 +Particular ears ought to know streets; tears could pr Books science 1.38 4417.02 1.45627188436706299 +Useful examples might understand evidently. Royal shops ought to leave in order. Also huge experts stay continuous, long organisers. Often burning services flee global circumstances. Fine, ex Books science 2.78 7923.96 2.61249443309046200 +Accounts accept Books science 1.24 4454.22 1.46853655921536677 +Small years turn as beside a problems. Famous, significant attitudes defend again subtle machines. Pp. double less. Human men appear in a regions. Exclusively warm Books science 1.75 3606.79 1.18914265043316062 +Certain, long councillors smile then fresh eyes. Lights attend initially after a preferences; national genes admit. Wide single plans improve never Books science 2.09 2209.49 0.72845904383276100 +Problems could not find small, late years. Demands might get only normal, available communications. Quiet mothers leave women. Fair interes Books science 0.21 8916.11 2.93960188337929509 +Marks remember Books science 1.41 1407.04 0.46389484135906840 +Things prejudice unfortunately. Available lives used to get for an readers. Roughly good articles might express open years. Black m Books science 9.38 11566.26 3.81334457287478571 +Small, stupid members lack hands. Literary terms would understand sure ordinary acids. Lovely, Books science 0.22 2581.68 0.85116843447228203 +Conditions must like most still desperate concessions. Parts shall not raise sometimes places. Local, prof Books science 4.37 214.32 0.07066035251313079 +Male, major regulations could get. Books may not bring. Upper, musical girls take well special, curious parents. Criminal, equal knees stop just a Books science 3.41 7411.80 2.44363755485639582 +Courts receive high male limitations. Political, little parents may establish tomorrow unique minu Books science 9.26 10412.18 3.43284952048418299 +Local, contemporary tanks provoke yet. Well red quantities should spend only deaf new firms. Books science 2.13 6975.01 2.29962983101256232 +Young officers depend very well unnecessary players. Personnel will consider apart types. Most universal courses could enable arrangements. Magic, equal responsibilities detect; value Books science 5.89 6948.34 2.29083685041567357 +Pounds realise fairly formal, casual residents. Good areas shall stick etc disputes. So small police find variable, certain programs. Results think children; dogs will take prices. Old, traditi Books science 44.25 3791.67 1.25009676564698863 +Left times used to tell trees. Right t Books science 7.96 2094.92 0.69068582347334800 +So clear employees could tell experiments. Hands would control demands; well ethnic sites afford then bottom programmes; times flow easily premises. Alter Books science 1.28 10461.12 3.44898482121203209 +However major deb Books science 0.66 2219.28 0.73168676336945170 +Thereafter strange rates shall not inhibit now on a heroes; eyes may not provide. Books science 8.37 11495.90 3.79014719324234879 +Due proposed concepts afford indeed yesterda Books science 1.34 10405.19 3.43054494851671946 +Earnings feel possibilities. Single, poor problems make full, sho Books science 2.75 17541.34 5.78330192213830518 +Direct schemes rival pa Books science 78.33 9776.79 3.22336425833730836 +M Books science 42.63 5228.32 1.72375389255063431 +Clear spirits shall not co Books science 2.11 1098.47 0.36216068227463034 +New, political bish Books science 1.33 1836.00 0.60532104896467022 +Professionally uncomfortable groups would not protect again there dependent users. Standard fields avoid likely families. Independent, intact fortunes work in the Books science 8.28 64.98 0.02142361751727901 +Future, solar deaths stand much confident, prime horses. New, other hundr Books science 0.22 7461.07 2.45988165511918956 +Acts will not reflect as with the problems. General governments distract new, soft fires. Useful proposals restrict hard trees. Large, black customs go official Books science 3.05 12762.28 4.20766705707016963 +Royal, considerable rooms reply then often c Books science 0.79 3487.40 1.14978029747243514 +Symptoms could not take else. Now rich Books self-help 8.22 4725.36 1.53069603755177003 +Normal sports will not afford from a women. Nearly past families would permit Books self-help 4.46 6912.33 2.23912593775928744 +There main prices could bowl acres. Radical, domestic plants take long. Fresh developments wave sanctions. British, able men cover goals. There other men Books self-help 7.22 5298.60 1.71638690482244922 +Results Books self-help 0.29 6602.84 2.13887217578942752 +About statistical blocks shall point so brothers. Even new affairs spend hopefully even old contexts. Possible officers wait absolutely with Books self-help 3.51 7809.11 2.52962181374665694 +Facts shall provide al Books self-help 5.02 1138.39 0.36876112342521194 +Men shall accept yet. Indians can continue obviously global, efficient times. Profit Books self-help 5.85 4729.95 1.53218288613311888 +Proper, mutual feelings would bring right over the days. Prices ought to see thus electronic owners; most surprising definitions might not see in part big lads. Responsible, tory doors read good, a Books self-help 6.84 4062.63 1.31601648192708015 +Early alternatives lie meanwhile european, new makers. Suspicious purposes speak new, overseas critics. Generally important police must refer approximately virtually other firms. British, appointed c Books self-help 2.07 157.85 0.05113269031937184 +Settlements can see so scientific sales; jeans ought to disco Books self-help 0.78 10137.10 3.28373262614193372 +Now christian papers believe very major, new branches. Annual wars include harshly so-called sites. Books self-help 5.23 8239.18 2.66893531470105824 +Much g Books self-help 4.52 725.34 0.23496094771145497 +Particular prisoners wait at a wag Books self-help 1.99 210.35 0.06813912834133586 +Good others run considerably excelle Books self-help 2.72 567.97 0.18398374482542681 +Very concerned shares must form also rather nice gardens. Quietly available games may see quite. Short eyes repay. As useful variables should not produce there. Managers use so also total versions Books self-help 26.11 239.20 0.07748457094959609 +Creative churches like. Walls objec Books self-help 6.05 3579.99 1.15967386770001887 +Now environmental examples enter banks. Royal, new attitudes go prices; almost living tre Books self-help 7.75 779.81 0.25260553207443365 +Hot steps help right able councils. Provincial mammals ought to establish from a others; forests used to offer true, open practitioners. Key theories could not imagine exact, other races. Books self-help 4.63 8643.42 2.79988164814865324 +Aware, a Books self-help 2.74 1189.77 0.38540475743604073 +Cultural notes ignore usuall Books self-help 9.32 5567.49 1.80348902138865697 +Positive, recent adults cannot tell fortunately laboratories. Frequent performances may get labour buildings; vocational windows will talk; similar seeds must replace better. Other merch Books self-help 9.69 10154.63 3.28941115678050571 +Tonight single claims used to compete seriously. Frequently magic advances concentrate very political men. Again damp types will apply also pol Books self-help 0.56 8790.86 2.84764220475738421 +Areas increase well final, peculiar findings. Fat possibilities will say now at all sure dogs Books self-help 5.11 3770.90 1.22151575499093605 +Clearly legal servants should not investigate however early difficult women. Increased laboratories tell home samples. Still wooden institutions avoid undoubtedly. Policies will Books self-help 9.11 9124.75 2.95579991125554742 +Physical, political issues must not increase. Teeth go there particular prices. Words mi Books self-help 4.82 1881.44 0.60945890956274278 +Old, acceptable respects imply around banks. Rights will not spare so existing reasons. Old eggs must claim. Patients might not stop there military, Books self-help 7.89 15529.28 5.03043310182334282 +National, dreadful opportunities give. Lucky, low rules should start away from the girls. Available words will not leave now. Stor Books self-help 5.53 6895.58 2.23370007419989892 +Dominant, useful restaurants might not say contrary eyes. Modest years may not confirm again just other stage Books self-help 3.87 12631.86 4.09186560365955223 +Various Books self-help 6.24 3437.60 1.11354916846292444 +There political deta Books self-help 8.83 4867.67 1.57679482221664051 +Other, established programmes used to avoid good organisations. Forward, simple changes might enter straight. Papers cal Books self-help 1.63 3028.98 0.98118401218606844 +Cards insist sad males. Instruments turn later instructions. Economic, white Books self-help 2.64 3883.30 1.25792572896557903 +Other, precious services can stem; grounds will set in particular friendly factors. Ports will provide. So complete moments diversify morally different, open pupi Books self-help 6.72 NULL NULL +Metres must not go more soft attacks. Northern, central changes see all right inherent metres; women shall reduce together always private efforts. Extra, secret dates ought to sa Books self-help 36.51 215.49 0.06980413960672434 +Outside, remaining problems must come only new politicians. Readers would not tell right, modern products. Particular threats become legally among a beaches Books self-help 1.38 24121.05 7.81358365427991146 +In order excellent words could go old costs. Surp Books self-help 1.45 3398.74 1.10096116500514307 +Logic Books self-help 1.29 3676.91 1.19106937194932846 +Sufficiently great tears may see. Much short standards duck over a pap Books self-help 8.57 1508.73 0.48872615689291017 +Again right years welcome to the months. Once competitive years could benefit great, social projects. Actually old expectations must not spin Books self-help 2.42 1824.90 0.59114378564346952 +Actions need qualifications. Expert sales see. Guests look evidently dead roots. Activities Books self-help 2.20 1248.95 0.40457506223870418 +Still social transactions provide both most existing vi Books self-help 6.50 2330.32 0.75486557431129919 +Prime even Books self-help 4.28 3438.17 1.11373380979002005 +Confidential, japanese reports discuss ever forms. Initiatives say now pregnant, sad sites. Neither round eyes may ask more w Books self-help 1.72 3385.13 1.09655244840554440 +Clever, informal negotiations study sharply with a leaders. Professionals come noble officials. Plans continue pa Books self-help 4.69 2768.44 0.89678672909573497 +British, Books self-help 1.52 4014.40 1.30039323419756920 +Highly other times could stay no longer huge symbolic results. Most narrow police chan Books self-help 7.99 660.44 0.21393775099477944 +Hands can ensure. Dead schools concentrate by a years. Increased authorities should not stop natural, following guards. Principal years might secure. Long, criti Books self-help 4.23 4140.99 1.34139980542043446 +Rights could not talk. Miners shall clear various outcomes. Relative, western forms locate communist, local prices. Items would not disappear probably likely women. Bare conclusions mark in gener Books self-help 8.57 3116.42 1.00950863962684053 +Other changes shall seek Books self-help 2.51 2862.54 0.92726874467415049 +So ashamed periods could give there on the operations. Potatoes must order very noble systems; labour years should not escape so formal, ready Books self-help 1.94 11014.72 3.56802196208166835 +Also crucial weeks will consider just then close parts. Long values prove then reco Books self-help 3.91 65.52 0.02122403465141110 +Sincerely important experiments should hear surprised, unchanged sorts. Else financial democrats will not start so major bodies. E Books self-help 1.90 5855.42 1.89675880614416367 +Cities practise a Books self-help 2.94 9089.11 2.94425496932977984 +Nearly northern eyes would not use further buyers. Ever independent advertisements comment also nice, old schemes. Firm members would restore as a doors. Problems Books self-help 8.02 14009.14 4.53801087906699247 +Essential, modern goods help friendly roads. Cultures Books self-help 1.13 8764.28 2.83903208813597843 +Gentlemen construct. Inevitable proposals tell more subject troops. Feelings used to come thus a Books self-help 1.73 8962.10 2.90311234660273887 +Miles kiss silently difficult streets. Less social rules see never Books self-help 7.03 283.44 0.09181532938943778 +Yards shall build gradually steep, possible players. Foreign, wild lines used to understand vital layers. Problems shall go likely, parliamentary rats. Suspicious, wrong thousands Books self-help 7.63 7823.86 2.53439981300044683 +Results Books self-help 9.21 3280.19 1.06255900829078431 +Smooth, othe Books self-help 8.62 11533.69 3.73613303141992873 +Available, other responsibilities ban common, english authorities. Participants save little for a years. Well local plans look. As entir Books sports 2.98 624.68 0.24146901355107034 +Now beautiful results scream just official payments. Carefully Books sports 4.89 12518.36 4.83895120778186737 +Agricultural elections go users. Popular customers could threaten upside down hard, able pages. European, interesting bases spend at a fingers. Books sports 2.47 7461.50 2.88423039734153702 +Levels should rethink really typically other women. Elections respond long numbers. Firms might sum nearly present, personal homes. Again clear Books sports 3.91 6886.83 2.66209266599525798 +Very social engineers ask facilities. Numerous, stupid Books sports 7.36 4152.23 1.60503759066587821 +Green levels provide. Students would agree. Very upper states get finally for a Books sports 1.29 4251.46 1.64339478189126194 +In order Books sports 9.54 5723.96 2.21258720433787633 +As specific characteristics contain for the hours. Free, double men avoid in the meals. Trying, potential institutions share above from the months. Contemporary problems could cheer only heav Books sports 1.58 1246.89 0.48198325271610120 +Grounds ought Books sports 1.69 6467.35 2.49994337066900616 +Completely particular voices shall not say straight. Used ideas must recall current colonies. New techniques could not make naturally old, great versions; great adults test Books sports 2.88 6653.24 2.57179884055600185 +Procedures order here shops. Late static sciences shall not see cultures. Polite implications cover underway. That is right communications might not say cool principles. Strange keys Books sports 1.34 2498.12 0.96564412520362400 +More big results develop again on a politicians. Characteristics live flowers. Children wipe perhaps appropriate roles. Wrong, external shows want somewhat little ways. Then difficult Books sports 3.64 4362.77 1.68642147699654727 +Basic, functional circumstances must Books sports 7.87 2947.46 1.13933575379592397 +Neighbours shall not represent overall dramatic trees. Random chiefs could not interfere basic, special fruits. A little poli Books sports 5.46 3974.85 1.53647164710487281 +Immediately impossible teachers cut kindly busy, national products. Important, principal communities could die all very video-taped words. Short children doubt windows. Sometimes russian developm Books sports 96.08 4160.79 1.60834644440858994 +Twice detailed customers know women; economic, intense values listen wide industr Books sports 0.74 6802.45 2.62947571753614401 +Sad, very sales could gather hence on a pounds. Issues see just within a things. Eastern directors put very in a initiatives. Books sports 3.99 5533.59 2.13899999791263899 +Sick organizations cannot cause to the situations. Direct nations seek to a genes. Able, invisible polls c Books sports 52.92 10879.04 4.20527479218581719 +Letters help; international directions should hu Books sports 37.74 460.35 0.17794752575436260 +Appointments might not hold to a tickets. Proper, private areas describe and so on prime, natural calls. Miners shall receive typically safe units. Little times will develop pointedly very mus Books sports 6.13 3351.79 1.29562884185557735 +Ministers prove perhaps social processes. Aggressive characters could get open signals. Products try at all public, loyal councils; wholly historical respondents see there from a statements. C Books sports 7.24 13142.40 5.08017283039890319 +Likely days shall get. Great users would love even. However acceptable walls Books sports 8.23 2406.70 0.93030587647013029 +Just average men might make so faintly free parents. J Books sports 1.41 9937.58 3.84135499725434718 +Papers conceive over immediate victims. Local, expert members add ill over immediate tiles. Profits pay even. Tall classes begin for instance grand fields; ru Books sports 0.25 3880.85 1.50013610366855243 +Great, reliable children see french, proper dates. Public passages like closely traditionally academic books. Values used to distinguish leaders. Much key oper Books sports 31.97 1293.62 0.50004665638396557 +Dual months should survive only large, political eyes; new, new merchants pass fairly conseque Books sports 6.26 4192.74 1.62069666369359458 +Conversely good eggs would not call too. Police happen present courses. Fine procedures finish well forward private Books sports 6.31 6912.27 2.67192645562313022 +Real, japanese systems would need downstairs for the phrases; level waters might not go about existing, little friends. Nation Books sports 5.90 2794.92 1.08037167086213344 +Devices take truly afraid, great men. Both true parties hurt even with a proposals. All internal candidates prevent more. Distinctive, prime women would say. Little, english departme Books sports 0.63 1050.56 0.40609221821766738 +Parents prevent alone little children. Cases might dispose again lives; very strange windows violate officially. Improved, cheap critics should alert plates. Expressions build c Books sports 5.56 4342.45 1.67856681484095121 +Wrong others miss less to the respects. Especially other standards start in order regula Books sports 7.53 11059.22 4.27492307108322362 +Adults will foresee most left, social children. Different eyes make personal counties. Readers would not admit more musical proceedings; titles take here away fast institutions; bird Books sports 3.83 10985.10 4.24627210853535058 +International, coloured contexts think. Relevant, british conservatives ought to happen ago. Perhaps human shops must see animals; rights must h Books sports 44.83 10933.78 4.22643444801245737 +Years should comment then central, internal implications; directly collective feet may find around extra, victorian crimes. Short Books sports 2.75 1868.42 0.72223463901372038 +So single phrases could not sum; desirable friends see times. French efforts think Books sports 4.59 4611.30 1.78249033455217177 +Central, visible moments Books sports 57.64 1362.54 0.52668756759280813 +Old, straight enemies obtain however years. Largely social questions disrupt never. Measures rule fine, extensive trees. Already economic friends would not show more beautiful engines. Systems ret Books sports 9.99 4644.12 1.79517685088812959 +Freely proud children cannot continue countries. Rates shall not look applications. Compl Books sports 4.13 886.97 0.34285677618843706 +Already secondary year Books sports 72.51 8152.72 3.15142033706550904 +Developers ought to recognize again. British, fast artists shall experi Books sports 79.00 2317.17 0.89569820408870728 +Paths check still international attitudes. Immediate Books sports 0.37 2211.39 0.85480912127281399 +All capital bacteria make jobs. Again appropriate eyes may not leave others. There fixed ways Books sports 0.32 7910.07 3.05762438371632671 +Papers occur critically relatively happy numbers; related, soviet genes experiment governments; voluntary devices Books sports 2.52 3864.91 1.49397452321775512 +Indeed similar changes might drink too national careful areas. Wise, good rooms give large opportunities. Various patients shall research directly plants. International hands can get pieces Books sports 9.31 3710.53 1.43429919134861534 +Here familiar rooms would not believe particularly new, fresh rights. Levels allow then wives; temporary, big ears may sound always others. Lovely, essentia Books sports 9.23 1808.93 0.69923887859854273 +Lines might clear too high eyes. Great women balance as the things. Natural requirements Books sports 8.76 5395.16 2.08549011197764081 +General, local thanks must soar actually about p Books sports 22.08 7752.94 2.99688604392750734 +Inc others look in the varieties. Cold methods write values. Partners will make often times. Democratic, dramatic personnel shall not see Books sports 3.64 473.00 0.18283736218488870 +Others used to coincide there so as historical sites; syste Books sports 4.08 4391.31 1.69745356646114923 +Poor, major pairs affect complex, redundant results. Different animals could find so great, honest designs. Dull, linguistic studies might not get more cons Books sports 33.21 1010.07 0.39044087615663959 +Open prod Books sports 2.74 12438.41 4.80804666844427361 +Bloody masters pull only women; shops take aggressively also legal cells. Continually underlying grounds would interfere. Entries shall not separate. Senior techniques see in Books travel 2.25 4171.41 1.64665291182793628 +Active, mi Books travel 1.40 12936.19 5.10652631399441219 +Voluntary others will imply again international, important birds; ill old publishers can get dark powers. Features stretch now progressive procedures. Tough n Books travel 1.83 3612.43 1.42599705573765030 +Cold terms shall comply only early claims; head, different politicians shall not commend good, foreign organizations; criminal, po Books travel 1.03 5504.86 2.17302872367020583 +Operations s Books travel 1.00 193.62 0.07643097580629212 +Applications might gather rather out of a problems. Scales could observe presumably for a directors; totally empty questions will forget. Just, symbolic question Books travel 21.48 5351.75 2.11258896173599765 +For example influential subjects shall work for example. Modules should persuade aside overall preliminary relatives. American, available reasons may use to the weekends; streets used t Books travel 2.18 6997.28 2.76215751673304277 +Similar sides assess more yet complete improvements. Bacteria would stay; general, curious trends used to reac Books travel 1.61 221.43 0.08740889873353613 +Communist, small cases may not turn other rules. Little, forward men should assist quite available technique Books travel 2.29 16204.92 6.39684871636659094 +Conflicts could give really sole institutions. Then advanced proceedings could not receive. Black experiences shall Books travel 1.91 5880.48 2.32130371144192077 +Leading players will sa Books travel 4.51 262.65 0.10368038320174892 +There european members turn; industrial, honest leaders cut exactly happy, consistent reasons. Incidentally european millions worry at first aware Books travel 3.81 2395.24 0.94551456714318326 +Deliberately ordinary procedures will not pay by a months. Feet reach very s Books travel 9.43 1776.74 0.70136335065629308 +Good, national parts remove animals; Books travel 2.57 3370.45 1.33047609960911726 +Odd, artistic databases open now; female, left days use all obligations. Simple, early sites may not hesitate statements. Left, free s Books travel 2.31 9717.76 3.83605970174234756 +However solid hours visit painfully things. Clubs must take most other words; officials will follow necessary developers. Alternative, great decisio Books travel 2.68 1892.11 0.74690534879063830 +Finally surprising cells cannot look better points. Elections might choo Books travel 1.98 3145.02 1.24148821160161580 +Right schools go now; average, invisible hands should get also good persons. Usually good ministers will make. Notes ought to stem average words. Heavy, certain suggestions summaris Books travel 4.55 337.50 0.13322721999082528 +Thanks could talk well individually national records; just simple officials go then encouraging, remarkable needs. Signals assess now. Upper, cheap pp. would not see. Hard trousers shall send whol Books travel 4.23 6920.66 2.73191197719023675 +Reports used to think characteristics. True types break extremely deliberately white tasks. Courses must cost. Economic, nervous resou Books travel 0.74 1273.19 0.50258833842998175 +Dear signals know finally. Positions answer payable payments. Superior babies can exis Books travel 1.78 16390.16 6.46997170964392568 +Horizontal problems continue members; modern, other interactio Books travel 8.51 2371.88 0.93629326978322569 +Open conditio Books travel 8.17 9456.37 3.73287670016189772 +Practical writers used to succeed recent arms. Books travel 9.48 10115.82 3.99319281934100804 +Members show yards. Economic stones get newspapers. Only magic views lea Books travel 9.23 1653.26 0.65261995176898313 +Investments ought to use still also professional developments. Only fresh visitors know steadily never main occ Books travel 1.37 4036.41 1.59336202383160616 +Conclusions might take on a ch Books travel 4.48 4341.46 1.71377969333738765 +Small, original things announce in addition at last other functions. Best political women make even old materials. Downstairs wet arr Books travel 0.34 8289.45 3.27223815926799005 +Again english deals cut for the cases. Yet normal systems reach biological, original reasons. So other remains spread steadily. Much inadequate members consider Books travel 1.92 7192.94 2.83939377706905721 +Later severe rules would Books travel 1.57 3713.31 1.46581916522705609 +Movements may describe quite southern, nervous views. Young notes imagine. Sensitive women might excuse then sales. Proportions may not exist only from a controls. Are Books travel 2.49 6651.86 2.62580389797976612 +That is fine terms know to the goods; useful colleagues us Books travel 6.31 6202.60 2.44845971767434933 +Yesterday long babies may not include as else able companies. Large, true d Books travel 4.19 1813.84 0.71600847617232157 +Words see low courts. National, democratic plants avoid. Days should go stupid, apparent days. Dependent hours should not want police. Also urban wages shall not define so great, typic Books travel 8.88 8312.77 3.28144366673520796 +Masses can contain as. Military men retain in a earnings; british, related instructions shall know different, precise needs; favorite Books travel 5.09 959.36 0.37870478746784635 +Behind relevant areas find then necessary papers. Copies might come envi Books travel 7.07 7437.38 2.93588581160107894 +Remarkably good bishops would deprive transactions. I Books travel 0.59 7014.30 2.76887611609376528 +Running businesses find emotions; Books travel 4.40 2300.61 0.90815962839434831 +Pink, central countries shall defend rapidly Books travel 6.87 6536.14 2.58012373828394893 +Local, conservati Books travel 1.68 8121.86 3.20608245616202735 +Strong women know also also obvious votes. Private, natural areas should play strongly for Books travel 2.11 184.12 0.07268087628062445 +Colours meet certainly hours; aw Books travel 1.63 5441.98 2.14820701228347073 +Too full weeks might obtain most today vital cities. Police shall take for example full sto Books travel 3.82 5904.69 2.33086054402259597 +Exceptional hundreds compare else then previous scientists. Rapid, popular differences get exactly now social persons. Naturally fundamental dreams hold on a changes. Brilliant birds pursue te Books travel 5.39 3124.51 1.23339194409935853 +British leaders can focus. Different workers cannot breathe only in an objectives; arrangements might enter predictably hours; reduced, effective phases operate ready men. Others say o Books travel 4.95 1624.50 0.64126701888917236 +Yesterday public notes work at least students; accidents might not apply today rural, subject premises. National, particular organisations could not endorse simply under a respondents. Sti Books travel 9.83 531.86 0.20995030881280099 +Maybe gastric variations will see as. However physical plants would not choose for example wi Books travel 6.36 1691.34 0.66765192965713314 +Little arts can grow directly rights. Full, slim argum Books travel 4.77 16542.31 6.53003251415238218 +About right clothes must get thoughtfully to a cases. Eastern improvements Books travel 98.75 2730.37 1.07780623598918408 +Countries want incorr Books travel 63.33 473.46 0.18689706541290708 +Fields would die clear horses. However new problems go nasty, smooth ways. Interested others go great societies. Familiar patients shall seem trends. Yellow, r Home NULL NULL 7995.48 34.64319649767261090 +NULL Home NULL 0.87 14048.70 60.87087637475838958 +NULL Home NULL NULL 116.76 0.50590328824138814 +Neat, desirable words make especially gradu Home accent 7.11 1583.88 0.73384072874422647 +Common males protest probably statements. Subsequent, main ways begin then titles. Rights come therefore interesting, ordinary thin Home accent 8.82 1429.40 0.66226730413099308 +Offers go odds. Black, certain readers prove again in a cases. Public, black things watch as else modern forces. Difficult, new crops comp Home accent 3.59 4707.69 2.18115934307012370 +National, round fields would not accomp Home accent 0.17 1970.93 0.91316811090730250 +More general applications work also moves. Final, equal instruction Home accent 33.79 1466.94 0.67966027642501678 +Severe plants filter fair with the days. Both great hills bring still. Military standards ask now for a conditions. Ago new proposals may like particularly men. Then alone a Home accent 5.54 6369.32 2.95102307649896240 +Present, good grounds fall students. Big, long nerves remain events. Important, black years must not use principles. Fatal mines cannot order hospitals. Forces apply elsewhere; now final members Home accent 5.37 187.59 0.08691389644741359 +Terms must work slow signs. Just american movements make surprisingly Home accent 0.26 481.20 0.22294880841460324 +Discussions could inform; legitimately potential miles remember again from the factors. Then administrative changes may Home accent 2.20 1475.60 0.68367261366705848 +Ago light fingers blame enough green, british years. Children go general stands. Economic, great numbers affect deputies. Purposes urge annually. Always electrical ways vote judicial, regular ac Home accent 6.86 11873.28 5.50110895256222018 +Days shall want later romantic, american changes. Reasons read; great reasons may occupy economically. Strong, new films go then objects. English relations would resolve over. New, crazy feelin Home accent 1.78 715.86 0.33167110139583931 +New, large words stop more strong cars. Back views leave other, young shoes. White conte Home accent 2.81 9585.07 4.44093918343840622 +Decades try then. Different leaders stray examples. Things would not participate too good, good messages. Exactly new thanks can forget; companies u Home accent 3.51 4955.85 2.29613643429241784 +Very afraid concepts will not disentangle with a days. Long-term, civil points c Home accent 8.15 3501.80 1.62244833189164095 +New measures shall pay under a agencies; comparatively heavy police shall beat similarly concepts. However japanese times cannot check like a police. Long, long-term auth Home accent 1.87 5547.93 2.57045798559357804 +Useful, n Home accent 9.44 3014.70 1.39676594498650122 +Days give briefly vulnerable months. Sexual feelings create just animals. Charts study; changes knock rapidly aware sites. Schemes include sufficiently. For example speci Home accent 7.15 303.87 0.14078855863039378 +Connections must not come right finally certain parties. Wild parties fi Home accent 2.55 1293.30 0.59920967149336320 +Little powers reach by a subjects; traditional insects make also others. Numbers shall make. Products take serious, military rules. Curiously economic methods approac Home accent 3.52 99.03 0.04588241998607265 +Old buildings must proceed; Home accent 9.33 595.01 0.27567907417866391 +Additional eyes give nationally. Territorial groups should talk previously strange differences. Small discus Home accent 6.07 18159.55 8.41365343691896978 +Almost busy pounds lose at last for an factors. Good mothers would Home accent 1.45 2292.51 1.06216203819318802 +Benefits might choose only by a directors. Continued eggs must not make much black, back arrangements. Living, Home accent 1.62 9494.68 4.39905983432661074 +Holes may avoid of course genuine Home accent 3.27 409.64 0.18979374455311320 +Supporters will laugh well indirect, old reductions. Men can increase critical words. Eyes ought to drift better parties. Other, social goods avoid costs; similar, substantial days learn; Home accent 63.79 5475.88 2.53707589572185700 +Main, powerful kilometres should like certainly political directors. Left families go tall, clear organizatio Home accent 0.18 11613.93 5.38094732857567124 +Promptly soviet faces could confirm now consistent new procedure Home accent 1.85 5675.68 2.62964690968951645 +Old events can try far natural genes. Primary months explain at all par Home accent 0.15 20335.22 9.42168135463177076 +Women should hear among a pages. Everywhere main techniques go just unlikely principles. Broad, willing differences can make also short, modern roots. Together sorry thoug Home accent 8.25 1632.64 0.75643213335415177 +Attractive, pale rights stop in a delegates. Answers go as; variable, alone roles ought to relax quickly concerned, detailed parents. Poor, physical matches would send as for a details; cent Home accent 1.45 989.82 0.45860180703437776 +Ancient periods will not see in a affairs. Fun Home accent 4.09 8014.62 3.71332082064806196 +Perhaps material e Home accent 6.64 2552.44 1.18259238684490834 +Here german thanks trust further remarkable towns. Other years Home accent 2.04 7200.88 3.33630011541261051 +Supreme others can decide. Unfair, short presents give. Activities give simply police. Dark, impossible Home accent 0.13 2033.98 0.94238033528498482 +Still different holes ought to enjoy early problems. Mammals see usually. Powerful, public Home accent 6.84 1085.87 0.50310353822353537 +Always potential wages shall not restart sometimes at the efforts. Mere, high weapons would not go there physical pr Home accent 66.58 7246.44 3.35740890118021093 +Boys ought to answer. International citizens call areas. All quick cuts might back most white, central amounts. Strong mice make on a lines. Cultures would dismiss changes. Left chil Home accent 5.45 18131.76 8.40077781891015469 +Most main firms would know highly for an companies. D Home accent 1.31 5733.85 2.65659814033265334 +New investors think especially secondary parties. Farmers detect adequately. Hum Home accent 38.04 1460.72 0.67677843605024781 +International, nice forces will turn modest ways. Trees might not deal eastern others. Responsibilities ought t Home accent 2.75 6806.25 3.15346077986677743 +Quite political women like home seriously formal chains. Certainly male lips Home accent 4.86 1551.13 0.71866705152980782 +Rules meet as; authorities shall not kill moreover near a Home accent 3.55 651.58 0.30188899540063836 +Also possible systems could go forward. Local, british babies d Home accent 2.53 2797.54 1.29615172379922932 +British results cou Home accent 4.30 118.60 0.05494956084366572 +Simply perfect shareholders come others. Other, tired eyes contact therefore educational jobs. Over cathol Home accent 7.12 11929.65 5.52722621010654933 +Enough labour losses demonstrate also quickly happy women; near available things might surrender also ge Home accent 1.26 1093.19 0.50649502882535352 +Royal children Home accent 3.70 188.00 0.08710385698658647 +Future, real fears mean far interests; ill, mean payments speak far so labour lights. Already other applicants might not go so powerful lengths; japanese, central modes boil. Old homes ough Home bathroom 1.70 19546.11 7.34362930968507144 +Also eastern matters should not enable now irish, Home bathroom 3.46 2574.19 0.96714369931910820 +Quite public shoulders help even ministers. Short, tall groups cannot overcome too other notes. Thus surprising reasons find Home bathroom 1.77 11046.40 4.15022051991445731 +In Home bathroom 0.42 1225.60 0.46046768804381146 +Necessary, p Home bathroom 8.13 5680.58 2.13423918027734537 +Letter Home bathroom 9.54 6366.89 2.39209131717465953 +Modern companies shall not become also old, grateful agents. Enough joint programs approve titles. Jeans will not fall already wrong teachers. High, silver children manage a Home bathroom 2.28 16790.19 6.30820820097611185 +Detailed, unhappy groups play old, human others. Well anxious councils will study whole, democratic employees. Educational, english customers get more. Explicitly cold deci Home bathroom 79.37 2249.42 0.84512502189907830 +Pp. may not record also human rocks. Extraordinary, industrial measures may not operate only out of a officials. Ready subjects show clearly new things. Projects should enable Home bathroom 3.56 11356.89 4.26687408752274959 +Here economic areas develop too sole processes; grateful, new children pass shares; fat, proposed aspects affect gmt on the terms. Years remind e Home bathroom 6.16 5399.13 2.02849617211813296 +Appropriate, active areas change alternative books. Clients will not look now only, other rates. Usually effecti Home bathroom 2.89 2344.36 0.88079473657179327 +Employees watch never at the imports. Cases resist actually reliable prices. Alive, var Home bathroom 7.17 2759.95 1.03693521182809843 +Very oral hands ought to smoke military, independent issues. Moving sons play. Patients contradict to a measures. Other cattle enable significant goods. Initial, possible groups let soci Home bathroom 7.17 3821.04 1.43559518172562445 +New sports will give now students. Scarcely free countries damage there prime, necessary members. Big units should not fill probably mental child Home bathroom 4.29 1777.37 0.66777207465602902 +Unions last moving pur Home bathroom 2.72 3881.21 1.45820153028110433 +Indeed political miles imagine. Urgent, able males can explain companies. Accor Home bathroom 5.47 2914.22 1.09489568036148517 +Almost other bodies call cars. So international benefits ought to suppose in a points. Officers can ensure also for a books. Carefully different police sleep. Irish, u Home bathroom 9.17 4471.44 1.67995564541989254 +Labour, japanese economies care more minor, great gardens; events may m Home bathroom 5.15 5956.38 2.23785943840600333 +Smal Home bathroom 3.40 1261.44 0.47393306168895686 +Free, sad bits might not speed then. Troubles Home bathroom 5.76 175.15 0.06580525094718797 +Hard players show empty troops. Expectations used to know even; alternative organs could not consume historical, direct practices. Material restrictions could count deep. Gifts could s Home bathroom 4.64 8640.19 3.24618824539756797 +Mere, alternativ Home bathroom 6.84 4069.67 1.52900745430912057 +Strong taxes represent nece Home bathroom 3.36 2436.99 0.91559656583378597 +Simply costly processes should not believe therefore by the weeks. Instead earl Home bathroom 7.28 419.52 0.15761700757844303 +Joint lovers can mention tomorrow minor techniques. Major markets may no Home bathroom 17.20 2682.86 1.00797188442005549 +Pretty figures ought to join that things. Extra authorities find dramatic items. Over mutual cases give for the time being as successful lines; permanent arms return publi Home bathroom 0.31 15228.27 5.72138240845865918 +Both long tories will not get together; problems seem by now special, Home bathroom 5.62 8655.20 3.25182762202741263 +Sanctions will know black quarters. Cent Home bathroom 4.35 2089.84 0.78516954404494038 +Comfortable clothes ought to carry violently. New, united services must look always. Common, recent workers could prevent. New, local languages need very often young kinds. Structures might Home bathroom 1.84 4089.18 1.53633751680400859 +Drivers might put Home bathroom 7.91 1583.75 0.59502749750276305 +Financial forces may bring yet. Unknown, expensive assets offer enough securities; female movements ought to grow great, aware modules. Normal contacts mus Home bathroom 2.10 4156.11 1.56148365123675362 +By now developing masses used to flourish subtle methods. Much Home bathroom 9.84 4755.08 1.78652145403342606 +Thereby social children should report to a days. Times meet anyway as a whole liable reasons. Physical, region Home bathroom 5.82 12047.28 4.52625911293770307 +So present rises l Home bathroom 5.86 3137.27 1.17869734307213477 +Philosophical, Home bathroom 6.72 3878.46 1.45716833336357782 +Single p Home bathroom 3.92 6593.22 2.47712530202694074 +Areas ride perhaps even leading women. High sides cannot get then throughout the officers. Long signs may not embrace to the friends. Very, tory Home bathroom 9.18 6130.98 2.30345804996968600 +Hi Home bathroom 2.13 440.85 0.16563085857874860 +Force Home bathroom 0.20 6396.38 2.40317094521024374 +Hard programmes make as other goods. Rational, similar computers could go to the streets. Options mi Home bathroom 7.10 4799.14 1.80307514719205068 +So straightforwar Home bathroom 1.16 1899.26 0.71356711912050371 +Properties go industrial troops; sweet companies would start more constant negotiations. Groups will protect. Public so Home bathroom 5.64 10621.64 3.99063480257316377 +Especially linguistic games cover to a officials. Minor, main days know completely variations Home bathroom 1.60 3572.22 1.34211152462782650 +From time to time successful books decide important, active elements. Parts will hear on a clubs. Firstly following supplies take barely upon a years. Other cases may find Home bathroom 3.90 218.22 0.08198699321550305 +Important kinds can catch again slim areas. Good, past men must Home bathroom 5.17 6013.16 2.25919213694315054 +Formal, positive soldiers co-operate long along a offices. Great, able details must overtake responsible, remaining papers. Lives would think acute, labour shapes. Representative Home bathroom 10.92 3002.22 1.12795798172233325 +Social Home bathroom 5.38 4680.62 1.75854623858650847 +Main forms matter constitutional, popular animals; ministers might not allow hardly. Officials will think so. Soon brief relations interfere for example old terms. Co Home bathroom 8.37 867.00 0.32573880999835553 +Probably awful sales require massively as annual notes. A little national devices arrest sharply short, grateful legs. Trees may protect immediately in a courses. Indians will not get i Home bathroom 4.33 1138.62 0.42778860881237321 +Military characters would Home bathroom 2.10 8317.61 3.12499236843185918 +In particular acute origins could like thousands; impatiently small stones might give away female, crucial models. Colleagues might accompany bes Home bathroom 3.25 4807.80 1.80632877821233414 +Afterwards oth Home bathroom 0.24 7197.60 2.70419568494136532 +Material officials tackle employers. Clear shareholders go very products. Areas imagine systems; superior, precise tonnes will make much minutes. Milita Home bedding 18.44 3038.10 1.25620354127751860 +Large tests complain dark, pales Home bedding 37.80 10472.58 4.33023668816435133 +Great servants deal primarily certainly possible gates. Problems ca Home bedding 4.62 4172.20 1.72513492476154936 +Usually large paintings might not go beautifully local appeals. Clothes bring partially different, very orders. Fruits provide except a schools. R Home bedding 33.55 1050.47 0.43435177709943549 +Well healthy Home bedding 7.46 10368.46 4.28718480945140073 +Conditions know both popular Home bedding 2.48 18121.95 7.49312325626349635 +Payable, mutual pictures will not help new women; mole Home bedding 49.59 591.36 0.24451747018527152 +Increasingly sexual Home bedding 0.50 233.74 0.09664758096777828 +Thus angry stations would not demonstrate forward; single, political winds must not accept then dark profits. Patterns used to know obviously. Wars use particular met Home bedding 64.50 744.66 0.30790445641937955 +Notes shall say slightly to a files. Important suggestions stay today acts. New, true powers make in particular; awkwardly left prices g Home bedding 0.79 546.70 0.22605130707232133 +About political men Home bedding 3.09 589.74 0.24384762727790521 +Yet personal children answer; sp Home bedding 4.17 1458.28 0.60297439194699971 +Sacred, other police run competent, poor solutions. Just subsequent lips allow far all small sentences; programmes used to develop with a conditions. Properties m Home bedding 1.39 2951.80 1.22051993454559739 +Attractive, dead situations shall enter also great, forward groups; thus compatible sections give still troubles. Cold, known waters can ho Home bedding 5.95 634.78 0.26247091403579318 +New, hard children say needs. Particular, horrible sports can clean. Corporate, adminis Home bedding 8.14 2691.36 1.11283235010455958 +Female abilities remove hard, happy customs. Really current shoulders lead to a heads. Vast advantages ought to explai Home bedding 2.45 2906.03 1.20159480499611843 +Clearly profitable ages cancel above evolutionary lessons. Steps would live better; labour women can bounce inst Home bedding 3.09 4184.78 1.73033654437554205 +Usefully clinical hours laugh almost attractive instruments. Responsible, obvious results follow even powers. Away big cups should d Home bedding 9.21 12113.91 5.00889919381098232 +Of course political others should turn social, low charges. Thoughts must not expand. Prime letters will not correspond alone Home bedding 3.60 3509.07 1.45094175984684579 +Immediately legitimate details may not laugh over bad, great publications. Pale conditions cost high, commercial arms; new problems should gai Home bedding 1.16 272.24 0.11256668709963190 +Criminal faces can exercise always to a members. And so on likely lines can know patients. New premises used to top also yesterday physical relatives. Organisational, alone operations Home bedding 93.25 255.70 0.10572767371207712 +Expensive parents could become very over the implications; prominent reasons bring Home bedding 92.94 4461.34 1.84468947922815077 +Just joint transactions might take now still national tests. Cells vary less so orange texts Home bedding 6.63 7559.57 3.12575576990069165 +Important, local transactions set overhead single prices. Available, white particles shall develop concerned, remote comments. Whole efforts m Home bedding 1.47 361.08 0.14930054135297930 +Eager, low years shall report clearly. Others should operate since a meanings. Directors would know holes. Poor boundaries hear early hours. Important countries make of course small, rec Home bedding 2.90 15764.84 6.51849769121275679 +Goods want special children. Personal plans remain. Payable, royal things go always concessions. Free, academic dogs raise still ra Home bedding 2.19 10328.90 4.27082741104682595 +Public applications will include less elderly, double businessmen. Federal cards impose partners. Places pay completely. Quite old ways deny ac Home bedding 6.98 7984.50 3.30145721843597883 +Good benefits pretend completely; s Home bedding 1.31 2239.67 0.92606608909944376 +Ways become just together close centuries; shots account also perhaps lengthy profits. Both eastern efforts might grab together tight countries. Police will express today for Home bedding 1.95 405.51 0.16767160331241453 +Electronic, long-term theories would give especially; elderly forms know yet later old risks. Different m Home bedding 82.96 15743.55 6.50969463226347981 +Double services precipitate finally demands. Authorities write early with a things. Full changes may not see in the doll Home bedding 4.48 1865.76 0.77146055731343376 +Critical, whole men forget in a industries. Alone lips look soon for a natio Home bedding 5.35 3628.30 1.50024137086245375 +Total, unlikely images get either open measures. Politicians visualise economically children. Able, ready states could not go in addition small Home bedding 1.42 334.80 0.13843420085570364 +Firm managers will not walk at a g Home bedding 3.23 1994.75 0.82479576510428565 +There controversial beings upset sure at a arms. Broad circumstances see pale memb Home bedding 0.56 8534.56 3.52889782931617102 +Difficulties will not feel most. Like things used to avoid both favor Home bedding 0.82 2845.65 1.17662868478205813 +Special, true decades cannot convert cool normal, old-fashioned books. Old ministers become. Substantial, economic recordings see particularly patients. Mass, absolute thanks could not su Home bedding 3.58 8483.58 3.50781845189793992 +Areas cannot get just. Horses achieve finally sad fans; tough examinations will not love also concrete mines. Experts shall l Home bedding 6.67 1746.36 0.72209065414087995 +Questions will encourage finally final, small institutions. Additional holes enjoy alread Home bedding 4.45 7157.46 2.95949000972719407 +Able, small executives would complete ne Home bedding 5.70 11277.99 4.66326025360996743 +Shortly official associations find however weeks. Empty subjects draw much linguistic, whole powers. Typical, payable feet shall sink also narrow boys. Permanent, i Home bedding 4.13 10215.08 4.22376474455520053 +Nevertheless left things must appear for instance again h Home bedding 6.76 6935.76 2.86782076740428637 +Enough lost problems manage before excellent champions Home bedding 0.97 425.46 0.17592059467164776 +Crude dates should convin Home bedding 9.48 2442.81 1.01006108181696956 +Personal, major issues say palestinian, german gods; angry styles keep surprising, pleased years. Authori Home bedding 8.78 375.34 0.15519681287090742 +Final off Home bedding 4.48 10411.01 4.30477852285167011 +Children used to solve all right required, military a Home bedding 4.08 5342.86 2.20918325682169878 +Able, red references might hire so direct children. Experiments ban too different, labour met Home bedding 4.41 1941.93 0.80295557845793480 +Then distant children plot. Previous roads w Home bedding 8.48 514.40 0.21269579725261037 +Powerful, happy companies seem also very national implications; children scan natural charts; really single subjects used to preserve. New re Home bedding 1.99 9617.02 3.97647693641971033 +Slight, royal projects will ask audiences. Elabora Home blinds/shades 5.27 7981.68 2.95699007390289399 +Years say much at a eyes; surely different theories may hear well powerful, free wars. Well little conservatives weave physical, fundamental servants; c Home blinds/shades 4.42 1284.84 0.47599742492224623 +States must not harm maybe late changes. Good, original steps must abandon incredible, useful neighbours. Sure annual shareholders could analyse absolutely patently dark Home blinds/shades 7.32 10474.36 3.88045856893354741 +Very able governments m Home blinds/shades 2.20 7440.10 2.75634977208368684 +Companies want as from the reports. Often different purposes will not work cases; principal towns guess Home blinds/shades 9.34 5385.32 1.99511102735147651 +Cells cannot give. Indeed english trees shall talk ever. In particular foreign things may catch too soviet, rich situations. N Home blinds/shades 0.28 8695.50 3.22144049719139513 +Tiny, exi Home blinds/shades 7.04 7025.12 2.60261124324411636 +Women must sleep in a scales. Agents can get generally extraordinary, general studies. Central systems profit; either comprehensive rivers use in the cars; cases shall ke Home blinds/shades 0.63 5940.92 2.20094534857964501 +Theories employ more specific offenders. Modes must see preferences. Certainly main studies see from the varieties. Pleasant elements Home blinds/shades 97.19 4156.26 1.53977853842294381 +Young opinions make fine weeks; copies would reply soon from the accountants. Interesting losses would go only slightly old families. Most famous patterns ex Home blinds/shades 2.76 8530.68 3.16037927900416200 +Industrial losses take letters; organic, likely yards could know all possible questions. Old studies retrie Home blinds/shades 9.59 8586.88 3.18119981329686010 +New, light associations feel on a officials. Potential, enormous customers find in the me Home blinds/shades 4.62 4568.78 1.69260570579703321 +Certainly tory systems read now in a prisons; evenings may seduce anywhere other months; new customers talk with the cells. Police lead more other exports. Young provisions Home blinds/shades 7.50 11150.34 4.13089032642781908 +Common, interesting figures would not see high; naked studies would get both. Changes might face over prayers. Tremendous, intact considerations shall choose just. Home blinds/shades 1.19 3490.71 1.29321080535345580 +True, impossible trees could get no longer exclusive cel Home blinds/shades 7.65 13982.16 5.18000074316711372 +Less whole balls should purchase often difficult specialists. Impossible, international intentions will not counter completely during a trees. Important sciences look initia Home blinds/shades 0.25 4673.99 1.73158307969266965 +National, electric sections must market in the decisions; b Home blinds/shades 3.94 13578.70 5.03053005338540591 +Thin, financial others can mobilize from a stories. Anywhere related others should remain following patients. Equations sh Home blinds/shades 5.47 1070.00 0.39640519027023090 +Steep, slow terms get. Affairs will decide upwards dominant courts. Familiar, serious years add Home blinds/shades 2.80 2331.69 0.86382618514130345 +Available laws get worldwide waste, new policies; then other societies understand by a interests; often local problems can help whole hours. Certain, m Home blinds/shades 8.96 9879.49 3.66007580675032100 +Clear accounts will not play even spectacular offices. Christian, impossible changes say for ins Home blinds/shades 0.25 7864.42 2.91354851071496196 +Rural, top years must accept again unusual shelves. Directors used to move later known, form Home blinds/shades 4.05 3163.86 1.17212198625081564 +Healthy directors understand at least young conditions. Excellent members prevent well meetings. Obvious Home blinds/shades 4.77 821.24 0.30424654061450881 +Thoughts must not achieve forward from the eyes. Powers seem recent Home blinds/shades 1.53 8071.29 2.99018808240767473 +Services must move amongst a bedrooms. Small markets used to advance in a courses. New levels could say from a centres. In particular present buyers must not transfer again. Indian, net courses s Home blinds/shades 0.19 3825.58 1.41727081102242049 +Different, upper days receive thorough, personal couples. Social, new girls must not prove strangely in a words; feet shall help however now full th Home blinds/shades 4.79 7716.79 2.85885570862188328 +Scarcely crucial groups may bring especially effective, important losses. Now new drugs wan Home blinds/shades 3.48 2706.56 1.00270507642784686 +Short candidates shed women. Involved, wooden needs might violate then long-term times. Students must not Home blinds/shades 5.18 NULL NULL +Only normal subjects might create over in the teachers. Main hours used t Home blinds/shades 4.63 2891.18 1.07110164299578147 +Bars like full, traditional politicians. Things used to show properly at the holidays; less specific relations may say possibly. Forces could Home blinds/shades 6.30 144.44 0.05351099596507678 +Prime, international results test ever conditions. Territorial users should love never barely emotional detectives. Firms resi Home blinds/shades 3.79 5465.05 2.02464877110871531 +Conditions make patients. New, various eggs will not watch appropri Home blinds/shades 2.22 360.68 0.13362189161370737 +Already early meetings cannot go animals. As comprehensive evenings w Home blinds/shades 4.11 511.70 0.18957059426287584 +Serious, free symptoms used to remember certainly able runs. Feelings shall pro Home blinds/shades 5.48 2291.60 0.84897395703108517 +Also long lines make further near a dogs. Rather foreign jobs can sit in the trends. Chronic police shall experience apparently diverse, proper years. Only notable companies migrate also years. Free, Home blinds/shades 73.55 6931.61 2.56796839339162169 +Complete costs become again industrial theories. Populations vary trustees. Countr Home blinds/shades 3.42 4143.26 1.53496240059723073 +P Home blinds/shades 2.11 8507.90 3.15193992364495091 +Minutes might not reply polish, main days. Main beans make properly agencies. As new years de Home blinds/shades 9.78 8403.34 3.11320335664060012 +Lives would look. Things exist for a patterns. Local, palestinian members should get problems; statements may not make yet nasty, specific men; numbers find clear women. Groups shall seem m Home blinds/shades 3.38 2112.47 0.78261128251416324 +Appropriate, extensive scenes stem openly now financial personnel. More concerned signs stay now members; also full days could prepare subtle horses. Ancient achievements say america Home blinds/shades 2.98 14371.92 5.32439596462480082 +Primary, occupational regions set particularly here prime ideas. Clinical, sophisticated minutes allocate just. Needs want interested let Home blinds/shades 4.77 5863.19 2.17214854910328515 +Large colours must win over; months assess extreme days. Blacks might signify then fully concerned points; here political potatoes might not die Home blinds/shades 0.55 3969.07 1.47042985845407977 +Sad increases ought to mean too levels. Organs used to present; other, sympathetic controls like always new interests. Other, small women deal in a edges. Outcomes run Home blinds/shades 8.43 7535.76 2.79178913703812636 +New parts come also old, tiny chains; responsible seats involve now properly small secrets; eligible chains get complete communications. Talks beat about married, liable books. Big, Home blinds/shades 7.11 1861.92 0.68978948772705450 +Social, central lights warn police. Home blinds/shades 7.78 6660.62 2.46757414805393022 +Subjects sha Home blinds/shades 0.26 360.45 0.13353668302140629 +Free, educational times ensure practically. So linguistic officers need. N Home blinds/shades 9.32 4744.02 1.75752724368764560 +Just possible women say. Reasonably strong employers light almost degrees. Palestinian, smart rules help really visual Home blinds/shades 3.71 8398.39 3.11136951954542476 +Labour taxes could get even lab Home curtains/drapes 4.54 24984.53 7.47827965622433549 +All real copies loosen more definite doors. Home curtains/drapes 9.49 736.67 0.22049741477429358 +Very, various goods should turn local arran Home curtains/drapes 3.04 3989.59 1.19414972919947050 +Unlikely sides sell sometimes friends; mutual floors used to say i Home curtains/drapes 3.70 11830.01 3.54091604348492652 +Roads help less functions. Relevant, increased procedures may not respond. All labour children ought to say workers. Given findings could decide thus royal shareholders Home curtains/drapes 4.28 5979.42 1.78973848785712263 +Weak girls swim; provinces may introduce. Nervous, green tracks say better british, public rebels. Houses must not s Home curtains/drapes 8.21 9746.45 2.91727235835165499 +Mainly alternative politicians will not maintain from a matters. Principles should not tell always details; suddenly democratic years formulate far. Western, wise years ge Home curtains/drapes 2.73 3116.99 0.93296623573285915 +Public metres want; characteristics shoul Home curtains/drapes 0.82 6428.18 1.92405971697478996 +Services decide only easy, single bases. Now british solicitors ought to transfer now over a drawings. Thorough elections run still religious, tough parameters. Complete, sole consequences ac Home curtains/drapes 4.49 6448.14 1.93003407238344634 +New, intimate hours go unfortunately forms. Subsequently experienced advisers must feed n Home curtains/drapes 0.70 188.16 0.05631937443350629 +Words might correct long old, major relations. Visible, desperate policemen may become extra agreements. General, other students include so Home curtains/drapes 3.90 10122.80 3.02992008671076475 +Centres look nevertheless with a advertisements. Naked users address to a reports. Im Home curtains/drapes 3.82 6381.83 1.91018640168464850 +Clear partners ought to take effective, black books. Circumstances become hospitals. Forces answer gradua Home curtains/drapes 1.32 1013.02 0.30321350280947356 +Certain, conservativ Home curtains/drapes 0.28 11983.75 3.58693294731893617 +Private years forgive then in the computers; more exclusive differences get sources. Minutes meet insects. Small circumstances will contact sudd Home curtains/drapes 1.69 2179.00 0.65221044265843012 +Known, possible years may approve. Forth wrong aspects see again local girls. Excellent peasants can run usually with a exchanges; Home curtains/drapes 3.79 4760.53 1.42490471711277482 +Prime, national features claim different, great views. Versions might not sign european Home curtains/drapes 0.67 9131.87 2.73331848324884729 +Free funds cause still new, Home curtains/drapes 4.69 8170.69 2.44562154278329893 +Years must not enable existing others; other, political ties like then short products. Quite Home curtains/drapes 4.35 696.96 0.20861156040166106 +Private parents carry really british dreams; writings look probab Home curtains/drapes 9.60 2216.28 0.66336895817119114 +Responses used to bring of course video-taped loans. Hot, positive systems would remember. New, personal words may not answer on Home curtains/drapes 6.31 2854.74 0.85447050898335328 +Germans will throw perhaps with a Home curtains/drapes 6.68 11036.19 3.30331269626550706 +Generally left questions bri Home curtains/drapes 93.18 2295.48 0.68707481730774354 +Particular, british wa Home curtains/drapes 3.20 6421.72 1.92212613300986409 +Democratic, likely appearances might expand both good, certain pounds; american values can pick. Only previous figures will not repa Home curtains/drapes 6.11 15070.04 4.51071016947234888 +Different, local measures say there political doors. Open assets progress minus th Home curtains/drapes 9.40 2024.63 0.60600496949037970 +Statements might not test financial, concerned authorities. United scenes back just bare publishers. More simple things could cope Home curtains/drapes 0.37 4710.47 1.40992093796661557 +Accountants look equally marvellous, british schemes. Things shall study tiny events. Both normal courses could appeal faintly. Then black practices used to die hardly. Advisor Home curtains/drapes 2.23 9441.66 2.82604371180834938 +Valid resources ought to say still tears. M Home curtains/drapes 1.25 8697.98 2.60344808904734832 +Electronic reports try in comparison with the problems. Germans might not go as. Common, social cups come sure about intact Home curtains/drapes 3.25 817.84 0.24479292722522739 +Outside mammals can ignore eyes. Amounts stand always that is ready notes. Structures remember most attractive issues. Subjective difficulties cause very. Adequate, di Home curtains/drapes 1.51 3062.90 0.91677621148164553 +Small females would allow topics; local, local tears find Home curtains/drapes 0.60 123.41 0.03693863732376175 +Problems must not hate there in a stars. Fully forward teams may work yet white, concerned personnel. Merely common years stem methods; measures could introduce more on a areas. L Home curtains/drapes 3.73 15982.27 4.78375557199933360 +Here other years may like later. Terms call yesterday also Home curtains/drapes 1.50 1201.77 0.35970947392089103 +Free, competitive aspects get even specific, medical times. Other, free days Home curtains/drapes 4.40 3406.63 1.01966023876708940 +National features sing then really magnificent values. Light, shallow advertisements should acknowledge. Possible, good forms should move anyway political, irish estates. Simply Home curtains/drapes 2.02 2017.71 0.60393369997996376 +Linguistic, appropriate degrees shout. Educational poles will study now in a names. Full arms look in a ways. Minute, modest systems deal unique experiments; automatically regular Home curtains/drapes 2.54 6407.34 1.91782196313128299 +Active books find; important, remarkable personnel may turn alone prices; public eyes administer different, financial waters. Obvious, weekly managers cannot make so. Proble Home curtains/drapes 8.93 25.68 0.00768644523518517 +Socially extra interpretations continue other men. Also odd initiatives must need now by a hills. So gross rules can divide. Significant, impossible parent Home curtains/drapes 4.37 100.62 0.03011721649393815 +Effects might tolerate reasonably. Comparisons take other, clear others. French, christian Home curtains/drapes 1.91 6527.01 1.95364115710692977 +New, different elections kill arms. As good as new yards would calcula Home curtains/drapes 0.59 4150.32 1.24225885469212285 +Events explore away. Unusual rights should affect so in a posts. New journalists might not find wrong scientists. For example tall authorities shall not con Home curtains/drapes 6.84 1245.00 0.37264892203292588 +Tall, whole women would not create. Still national hands bear around flat, poor attacks. Fiel Home curtains/drapes 6.19 2226.86 0.66653572571746292 +Months shall not find also intact forces; super ju Home curtains/drapes 0.99 6731.10 2.01472864184403808 +Superbly loyal police would contemplate twice sure nights. Even succ Home curtains/drapes 0.44 49.08 0.01469044907098474 +Legs solve by a women. Early, early weekends neglect again loans; proposals Home curtains/drapes 57.92 10980.48 3.28663777944104577 +Likely, normal policies believe very children. Twice old knees should suggest with a details. Lives take students; questions will not look as deeply ready areas; valuable members wor Home curtains/drapes 5.17 249.22 0.07459563401529782 +Budgets keep so lesser women. Stairs determine Home curtains/drapes 1.55 4402.52 1.31774645158907378 +Di Home curtains/drapes 6.03 5657.98 1.69352622319988272 +Particularly old assumptions might learn repeatedly fine sessions; payments compete more bad times. Days will plan formerly; all right simple jeans reject weeks. Today national representati Home curtains/drapes 24.89 14029.64 4.19930138354218335 +Goals commit then obvious tracks. Excellent days k Home curtains/drapes 6.14 1920.32 0.57478327546848854 +Human drinks Home curtains/drapes 0.71 1522.69 0.45576609404844651 +Dead, obvious terms would serve more through a forces; worthy, possible arms decide for the falls. Rules Home curtains/drapes 2.34 14312.02 4.28382234948889629 +Small branches cause smoothly right duties. Outstanding ministers give real policies. Increased, japanese settlements used to protect electoral, large offices; clouds Home curtains/drapes 3.90 15202.77 4.55043843567430089 +Specific, small functions make about a children. Other, hot notes request however new things. Very slight eyes should want always serious, normal Home curtains/drapes 6.32 1409.34 0.42183857974127210 +Somehow surprising officials eat important cells. Mature police operate close, permanent flights. Old, fine engineers will pay away fingers. Hardly cultural activities watch gay, new Home curtains/drapes 0.25 6118.86 1.83147516712481033 +New, perfect clothes let. High centuries could go months. Part-time, legal things think even there new systems. Aware losses come yet that wide functions. Big, british ears send please economic hee Home curtains/drapes 7.09 4208.63 1.25971199416500631 +Less than dark patients teach however national senses; as positive problems can take instead liberal collective sectors; urgent resources raise so southern motives. Private p Home curtains/drapes 0.67 7346.83 2.19902673081057097 +Still available arguments Home decor 6.57 7479.82 2.46562464048976131 +Then adequate experiments ought to need pp.. Able unions could need please on a countries. Women continue previously british ways. Home decor 0.96 3319.93 1.09437141705297364 +Now imaginative boys shall look. Experiments tell main confl Home decor 3.59 1502.18 0.49517395103771343 +Independent, limited numbers claim nonetheless to a firms; never managerial sources would want only special terms. Changing, present homes shall suffer Home decor 6.24 1843.18 0.60758013225691504 +Fre Home decor 2.65 4396.90 1.44938046393755886 +Wonderful, brief ships continue; less vital o Home decor 9.80 3685.64 1.21492292594937898 +Perhaps spanish Home decor 7.44 2152.90 0.70967527139829663 +Regional circumstances see really matters. Again sexual years secure adjacent trials. Old animals will solve new, necessary eyes. Level views migh Home decor 7.80 157.04 0.05176617800194552 +Old fruits tak Home decor 2.26 7882.54 2.59837601087274335 +Parliamentary, favorite months escape almost necessary, environmental beliefs; closely high doctors used to run far exact contributions. Kinds accept never european trades. Sorry, great tho Home decor 2.64 8778.45 2.89370100153577829 +Much red years would not repeat by the others. Particularly environ Home decor 1.45 2736.60 0.90208432705122327 +Sol Home decor 1.01 9042.00 2.98057680523173309 +Schemes wield usually other Home decor 1.43 5016.00 1.65345866567599792 +Helpful, very colleagues shall provide members. Concessions go other, tired eyes. Accurate windows ride slowly different hours. Speciali Home decor 1.48 2381.42 0.78500389465991526 +Frequently small crimes spend as primary regions; exactly small students simplify very. Early workers make interpretations. Late direct pensioners ca Home decor 2.82 6192.37 2.04123361993063780 +Maps form houses. Whole assumptions used to know for a premises; black titles Home decor 5.19 6005.87 1.97975633899990144 +Contacts choose to the governments. Over angry contracts could sell as yet national technical tables; violent, toxic patterns cannot express solid crops. Feet shall use Home decor 9.88 1269.31 0.41841140728253607 +Formerly prime men influence incentives; new bars support children. Machines end certainly so economic drawings; other, christian eff Home decor 2.26 5503.23 1.81406765006142784 +As Home decor 2.03 7855.62 2.58950218565743277 +Long-term st Home decor 8.22 2874.12 0.94741599286138340 +Contemporary feet used to go still political, late lives. Statutory, scottish genes must smell. Good lips establish quite. Old women must avoid with the places. Too wet l Home decor 4.58 710.24 0.23412130835520749 +Citizens can keep for the most part at the things. Branches visit terms. Available, slight problems may avoid. Problems help more. Social years feel inherent acres. Individuals use Home decor 49.10 5668.87 1.86866870536098372 +Workers shall not control never on a studies. Sophisticated activities go separately according to a bodies; co Home decor 40.34 2145.78 0.70732825670539131 +Prematurely other systems assume nearly under w Home decor 0.88 9056.13 2.98523457455908593 +Always cool temperatures meet there social grounds. Threats h Home decor 5.44 3350.86 1.10456708621751882 +Too complete events try environmental, national topi Home decor 3.31 7994.82 2.63538764145131214 +Fresh, beautiful functions give empty, fast origins. Sons get other companies. Lights say delightful, native services. Small, soviet things could go already also dead systems. Medical, comm Home decor 34.78 11689.03 3.85313555559144935 +Resulting, distinct clients shall tell intellectually difficult gardens. Villages turn then by a things; fresh, supreme powers succeed here. Historical hands st Home decor 4.30 269.93 0.08897888708650760 +Possible shoes render undoubt Home decor 8.28 13638.47 4.49574290431860593 +However old figures ask only good, large sources. Yet naked researchers shall deal to a women. Right, common miles describe there also prime bags. Readily significant shares Home decor 7.78 NULL NULL +Relatively regional months wish then needs. Eyes follo Home decor 66.29 7883.31 2.59862983128194800 +Deposits shall leave more skills. Close ce Home decor 5.30 5555.19 1.83119558312931557 +Regular findings put. Little, national cattle should say most mothers. Asleep eyes stay over thoughts. Western, golden walls might not move distinct, small boxes. Swiss, go Home decor 3.83 3030.40 0.99893164682307498 +Gentlemen work always. Religious, spiritual variations think fairly so electronic resources. Diplomatic, civil others split both mathematical, new contacts. Ultimate Home decor 9.53 6205.11 2.04543319397384199 +There final techniques wear so old winners. Old, particular prices will return especially motives. Around early members shall pay systems. Unions call rather. Else old ter Home decor 2.10 13195.83 4.34983242908439067 +Similar, ready forces play often arms. Marrie Home decor 7.68 7302.41 2.40714375893522009 +Nearly delighted services know then eventually political p Home decor 0.48 4915.69 1.62039278873142867 +Top modules ought to go. Funds shall offer in Home decor 4.71 13454.30 4.43503367735338493 +Important rights justify now still e Home decor 53.89 3370.57 1.11106422941936768 +Fields divorce hardl Home decor 1.25 14250.34 4.69743783130568185 +Able, assistant positions should die Home decor 4.24 3308.46 1.09059048186650958 +British, electric ye Home decor 4.13 6855.95 2.25997407076183372 +Immediate designs reward more speedily expected things. Good, happy feet create interesting, political signals. Still general stations help. Remote, flat ideas ma Home decor 0.10 6799.02 2.24120784232544325 +Sa Home decor 2.03 474.81 0.15651489414864844 +Minutes must not reduce in addition conditions. Australian, likely methods miss on a grou Home decor 25.40 111.84 0.03686659034473756 +Quickl Home decor 9.23 2919.06 0.96222987492587290 +Abroad great methods can call all labour main clubs. Minerals may make often countries. Apparently good pairs used to write terrible accounts; able funds close again with the times; earlier average Home decor 4.93 5327.91 1.75627570961758494 +Minor, usual members come more before good waters. Circumstances cannot take interests Home decor 0.15 15519.10 5.11566793829592889 +Present, responsible rates contribute at all records. Eyes ought to wait political, national awards. Politically int Home decor 0.18 20899.05 6.88909795193300723 +Nations realize on a shadows. Managerial, disabled systems stay between the councils. Capitalist girls might live Home decor 4.02 1089.18 0.35903391337340180 +Military issues face rather once previous thanks. Then famous sources ought to transport boats; readily impossible requirements trust again with Home decor 5.27 7325.56 2.41477485305611310 +Private, direct rates increase furious meals. Italian values buy for instance random members. Available reforms work financial, impossible adults. Immediate, good experimen Home decor 6.40 7796.60 2.57004701611034397 +So far conditions may r Home decor 8.95 1175.16 0.38737609361160401 +Suspiciou Home flatware 8.91 11913.78 5.12961692885790892 +Material, rough relations think cities. As popular studies should not ask at a boo Home flatware 0.28 1925.64 0.82910676064909237 +Real times could cultivate honours. Great carers enter like a drugs. Sufficient years read o Home flatware 3.21 32.10 0.01382102938079593 +Long, other grounds give now clinical, essential areas. Possible languages make. So similar costs would say. More similar propos Home flatware 3.20 180.81 0.07784985427855798 +Present variables shall raise royal, american structures. Home flatware 1.03 26390.07 11.36255242464987910 +Remarkable m Home flatware 20.08 15671.25 6.74743946055445923 +Changes like old, perfect streets. Thousands say. Whole factors work particular Home flatware 1.83 3396.31 1.46232088150439278 +Police succeed schools; supplies calculate far countries; new words move shares; officers must complete years. Asian things may bear warm things. Aw Home flatware 6.66 2788.28 1.20052647357899259 +Suppo Home flatware 2.16 18092.16 7.78979049601435527 +Streets will marry. Agencies tell regularly students. Years study here colonial, economic transactions. Cards shall not hide of course inside technical sons; else environmental Home flatware 58.71 3036.50 1.30740048955722201 +Early, particular conditions fulfil just women. All new sales might not feel large, active books; current children should take. Generally di Home flatware 14.12 22.62 0.00973930481600012 +Foreign parties could not keep ston Home flatware 1.70 4789.08 2.06199424881564327 +Patient Home flatware 1.87 9772.43 4.20763371189319384 +Years know more medical citizens. Then comprehensive observers come finally by a processes. Small voters must waste others. Statistical levels study. Ex Home flatware 0.33 741.75 0.31936911349549462 +Arrangements keep simply close large terms. Projects might not live true, easy others. So new years take labour members. Original towns travel away away americ Home flatware 9.19 2252.25 0.96973250538621876 +Possible, thick acids shall not go in a c Home flatware 3.98 5764.14 2.48181770389473594 +Random influences can force low for a subjects; young days will not travel historic hills. Unlikely, huge guards arrest now by th Home flatware 3.46 5434.00 2.33967207648738495 +Domestic, new tasks show here very various farms. Internal, old homes used to impose long traditional, high Home flatware 1.93 627.94 0.27036689063479730 +More special scots ought to see just on a pupils. Grounds might shut complex writers. Empty, actual eyes may get little wrong, odd words; social, full tact Home flatware 3.31 2123.58 0.91433213621403771 +Legal ci Home flatware 4.71 5052.16 2.17526641110535642 +Hom Home flatware 8.19 3362.38 1.44771192428039261 +Leaves cannot lose colours; european, dynamic sentences will Home flatware 96.77 1428.58 0.61509178046160258 +Further o Home flatware 5.51 11480.35 4.94299858728412768 +Thus internal planes would not apply never rather than a Home flatware 2.06 4826.77 2.07822211789234727 +European seconds wou Home flatware 5.97 12128.66 5.22213601899328053 +Labour, likely area Home flatware 84.74 7106.28 3.05969173421066874 +Particular, healthy talks get written, apparent months; then great attacks used to secure characteristically to a agencies. Accounts answer prod Home flatware 3.87 179.28 0.07719109493423967 +Yesterday angry obligations defi Home flatware 3.77 1418.04 0.61055366053407644 +European, rigid voters believe in common including a meetings. Complete trends broadcast directly; Home flatware 2.19 10595.74 4.56211943461914690 +Likely, odd offences shall ease enough true, chinese eyes. Other indi Home flatware 4.09 3818.90 1.64427193465176194 +Left, white ways might intervene es Home flatware 9.19 416.05 0.17913517987165560 +Later substantial changes give wisely. Minor taxes would shed forward reasons; yet long shareholders will live close small constitutional bags; supplies rea Home flatware 3.08 1033.24 0.44487353262970659 +Rather inc researchers might not answer sure. Most actual lives Home flatware 4.89 317.32 0.13662582688829168 +Forces used to adapt in a musicians. Rather political t Home flatware 89.07 4073.22 1.75377237677400555 +Other, white years get meanwhile plans; more royal sciences would not contain triumphantly splendid specific concepts; free months Home flatware 1.62 21553.63 9.28016677547677492 +Financial, black securities may support vague, late offices. So marginal incomes make on the men. Hotly close occupation Home flatware 6.87 280.44 0.12074671275857973 +Actively fierce lines should not feel quite confident new rules. Arms pay long settings. Wide, black women should pick real talks. Important friends make today between the revenues. Noisily expe Home flatware 4.53 8713.76 3.75181099617458879 +Brief regions ought to inclu Home flatware 4.98 5812.86 2.50279466811381312 +Forward general regulations can begin forward women; galleries consist typic Home flatware 8.74 2672.21 1.15055118136002115 +Uncertain, statistical jobs walk there; agreements show to a rights. Useless years may not resist locally only marginal experts. Concerned, Home flatware 0.14 7564.70 3.25706981174164905 +Beneficial, moving years ought to see difficult, political stocks; attitudes can say british questions. Upper, educational chapters should end then back lives. Workers talk there in a boundaries; pro Home flatware 2.02 609.71 0.26251775151916148 +Busy, new things go satisfactory services. Now old years must take. Scottish procedure Home flatware 0.85 2855.80 1.22959799706158888 +Mislea Home furniture 1.06 2910.97 1.06321050660037366 +Papers check other, industrial boards. Violent, social things give cars. Local councillors give ther Home furniture 3.38 3631.97 1.32655048442868154 +Dutch, busy firms must not return thereof full, naval plants. Parts shall get ashore early politicians. Good organisms try rather also close boys. Positive, big ingredients foster greatly local grou Home furniture 1.71 1113.86 0.40682922011628158 +Arrangements will trade however in every negotia Home furniture 3.24 15049.37 5.49667234692094570 +Black, perfect visitors should test more low english interests; about major wives believe examples. Other, available gro Home furniture 0.66 10969.33 4.00646757141663321 +Marine, new services shall reach more more significant elements. Late, solid rights would like also. Notes complete elements; continually personal armies will compare clearly curre Home furniture 3.59 965.34 0.35258337613977633 +Ways become worldwide specially common gene Home furniture 8.57 791.04 0.28892157567448637 +Very likely areas should k Home furniture 2.37 3579.84 1.30751038311912580 +Arms fail other faces; leaders could arise good characteristics; gol Home furniture 8.75 2288.09 0.83570814128872814 +Stones tell. Still brown relationships put initially long r Home furniture 9.54 5599.90 2.04532252682488396 +Private, young standards find even so in the women. Sheer, expert classes cannot present men. Small, sure enquiries must support mildly p Home furniture 4.99 2942.39 1.07468643184775984 +Authorities used to consider; general weapons seek particularly economic papers; much american walls Home furniture 1.27 2216.17 0.80943988718968251 +Severe, likely areas make on board formal, new conditions. Democratic, individual numbers should not fight workers. Poor options think. Independent feelings squeeze only ideas. Thin prob Home furniture 8.47 3094.07 1.13008644271738222 +Adults might not surrender doubtful, upper industries; earnings insist m Home furniture 1.61 6969.96 2.54572692352870019 +Shareholders mean; more very teams believe necessary, charming words. Courses would not suggest as popular, similar assets. Subjects must make on the things. Liabilities used to get very to a lines; Home furniture 8.45 3751.07 1.37005088853319121 +Directly high lines move calmly also international files. Pounds cannot ensure creditors. Similar, favorable colleagues could gather written police. Free days might provide so. Probably other rock Home furniture 6.83 5386.33 1.96731764601379975 +Streets know half. National, Home furniture 0.39 9772.83 3.56945469558921243 +Soviet, evident ways change able, huge woods. Smart sales ask sales. Thus possible transactions can want below effective, available families. Also external Home furniture 4.84 145.90 0.05328890813474358 +Usual tools happen little young children. Dramatic, Home furniture 1.68 11143.74 4.07016954857756966 +Judicial operations cannot kick currently h Home furniture 6.22 9022.42 3.29537293031578591 +Too young things leave individually skills. Contexts suffer enormously so romantic Home furniture 29.66 20545.03 7.50392197598047208 +Superb lights occur with a standards. Bright services specify at the sides. Then urgent versions get earlier prisoners. Available heroes would not believe civil sides. Banks could t Home furniture 0.12 16046.32 5.86080104441877032 +Royal, military notions will not find very very wet acids. Funny actions take western, remaining homes. Great patients will replace simply. Signs can think equivalent reasons. Campaigns Home furniture 7.54 1334.66 0.48747480555940278 +Yet huge priests think today unlikely, absolute things. Whole, modern changes might not manipulate most only, desirable companies; accused, particular girls may take serious, central hours Home furniture 0.52 10920.86 3.98876425834404225 +Local blocks shall not get natural things; already post-war patients may exploit british, sexual grounds. Easy centuries would not Home furniture 3.75 2996.52 1.09445701853270617 +Ago new arguments accept previously european parents; fo Home furniture 3.03 6882.58 2.51381201747788529 +Walls s Home furniture 4.80 1253.04 0.45766369738971278 +Late general supporters see more informal, blank employees; very similar methods shall help complex, likely schemes. More than new groups reconsider unanimously. Physical incenti Home furniture 37.53 2259.23 0.82516723732184192 +Mountains ought to join pressures. Bright countries used to pay there owners. Imperial issues might establish thus calmly senior members. Just regular Home furniture 7.01 10713.70 3.91310058316108488 +Contacts open considerable, suprem Home furniture 7.01 1997.51 0.72957592109822925 +Effects must quit about small values; full paths must get. Problem Home furniture 1.87 4806.19 1.75542575317425115 +Political girls used to ask hands. Large-scale, chief areas can produce including the children. Sufficiently new areas will Home furniture 2.26 3164.50 1.15581048521176187 +Now late makers used to Home furniture 0.85 7607.78 2.77868601459451341 +Greatly commercial animals used to live now as wide personnel. Enough hot wars keep. Min Home furniture 4.37 894.54 0.32672419385094943 +Better high children Home furniture 4.48 4768.72 1.74174010966630844 +Thus light firms expect anyway in a toys. Laws used to ab Home furniture 2.06 12227.85 4.46613279873491621 +Widespread others hold quickly new teachers. Societies wou Home furniture 3.01 1696.19 0.61952099444188288 +Hot, small levels ought to arrive only there other features. Often irish columns used to spend now new natural months. Once british flowers shall penetrate funds. Home furniture 5.70 20519.61 7.49463750685925767 +Electronic organizations transfer still natural, whole posts. Plants ought to curl just animals; already huge women can dream eventua Home furniture 3.59 6214.52 2.26980798753616633 +Increasingly other policies happen previously under a targets. Efficient, experienced points will see mostly even english machines. Fine states must remedy also good thoughts; normally clear years i Home furniture 5.85 9156.23 3.34424605435629337 +Natural costs assist during the types. Sometimes possible concerns make as real, right forms. Home furniture 6.28 1707.15 0.62352405429902331 +Therefore early eyes stay recent, expert studies; varieties halt in a parts. Unable i Home furniture 7.52 742.08 0.27103929368492471 +Funds drink much common months. Royal, long trees will expect sometimes front coins. Old ears can allow very similar, short members. Even public rules act common, open Home furniture 17.29 6237.51 2.27820491692628117 +Intensive minutes might see like a boys. Questions might know more young communications. Ready, southern others may result. Lonely, trying seeds love probably good farms. Home furniture 9.12 11445.81 4.18049840724968750 +At least competitive notions may not convince white, familiar principles. Valuable, fat books convince further cases. Yet ordinary cities cannot need so as. Ri Home furniture 8.51 1332.65 0.48674066775713524 +Women should not knock doubtless details. Sure northern products must go very cruel, other tickets. Poor, physical objectives highlight only by the discussions; now slow crowds must Home furniture 0.77 87.87 0.03209387496778559 +Little, evil parties would not act subject Home furniture 7.63 1108.98 0.40504683580032854 +Easy, philosophical levels must Home furniture 2.32 3778.34 1.38001105662664191 +Now additional reasons hate. Original, use Home glassware 4.41 6349.14 1.56441659290736902 +Jobs notify about future boxes. Now main policies will think above offers. Criminal men used to think soon national women. Sure talks ought to appreciate there companies. So appropri Home glassware 1.19 7756.30 1.91113826747676477 +Seats will cope similarly new shares; massive deals explore semantic, important thi Home glassware 1.53 4412.81 1.08730838906490754 +Powerful hours take worth a authorities. Respondents must generate aside c Home glassware 31.97 10526.17 2.59362921714811148 +Unfair, possible hands will not arrive surely tight russian employers. Really necessary walls should decide varieties. Talks would raise probably moral meetings. Bright, necessary Home glassware 1.54 3919.44 0.96574291493097623 +Old Home glassware 1.47 1351.66 0.33304657512185499 +Conditions criticise enough more particular shops. Be Home glassware 6.38 1038.40 0.25585987867254652 +Countries ensure in a christians. Expected ends used to run high holes. Broad, unlike women specify therefore. Lit Home glassware 2.94 153.37 0.03779009013097887 +Onc Home glassware 4.53 1345.23 0.33146223477144621 +Western, complete meetings follow also educational shareho Home glassware 7.67 2508.40 0.61806521539119384 +Similar, low sites remember peaceful days. Faster permanent views give then churches. Others make well public processes. Eventually other schemes can trus Home glassware 0.29 105.75 0.02605660840680065 +Statistical bedrooms analyse there good, little residents. Home glassware 8.08 5239.63 1.29103533906879324 +Less than outside students go more. Military views should not let more different, big steps. Average, black animals ought to begin automatically with a notes. Needs Home glassware 3.76 13328.83 3.28419956341197821 +Wide, great premises mean ever severe courses. Used ministers face there about a things. Home glassware 0.83 1275.20 0.31420696964872045 +Faintly actual prices may not wait dramatic terms. Others shall see shortly priests. Very na Home glassware 27.85 6812.75 1.67864925695915955 +Agents invest often things. French cars ought to get locally distinctive, local powers. More american entries compensate only Home glassware 6.43 10473.16 2.58056764918929822 +Again other wheels ought to find on a employees. Developments make really together new groups. Drinks would not assess bright women; special, australian t Home glassware 3.25 516.63 0.12729669599248624 +Words visit authorities. American occasions must need available, pure c Home glassware 5.43 5888.06 1.45080731627183575 +Purposes look events. Words convert over complete sites. New notes tell up a Home glassware 9.93 9702.28 2.39062421383578063 +Free kids would become only Home glassware 1.05 8484.78 2.09063441964873770 +Interested, square savings change off Home glassware 2.10 8572.37 2.11221643695702771 +Exactly single cities used to deserve ago false services. Suddenly psychological managers could sustain far together big changes. Parents should r Home glassware 0.64 2997.09 0.73847754600414333 +Heavy, desperate standards could produce still fine, important weeks. Accordingly Home glassware 9.90 11317.37 2.78857946368674669 +Long, surprised sections keep positive sports. Strategies go northern, precious forms; readers emerge about reports. Large, unusual legs might show affairs; as usual ac Home glassware 4.43 12838.25 3.16332154022324760 +Red rooms could not apply Home glassware 4.96 1551.75 0.38234838860759250 +Present materials would say real, rare relationships. Particular conclusions contribute well to a hand Home glassware 4.07 8454.05 2.08306260332400026 +Separate moments come months. Avail Home glassware 0.58 5564.41 1.37106054264667234 +Professional, local chemicals can feel eyes. Familiar shops bear early in a accounts. Western arrangements ride reserves. Sorry, scottish ministers might not keep constantly w Home glassware 6.13 5921.40 1.45902223186788996 +Rows come Home glassware 0.29 840.56 0.20711246111035795 +White, local attitudes ca Home glassware 1.74 1012.36 0.24944366985067333 +Seconds may make ahead quite little lips. Young, criminal consumers s Home glassware 7.17 1471.96 0.36268827716760552 +Recently nice particles hear above in a candidates. Human errors register. American, old days live. Home glassware 8.16 528.66 0.13026086619706129 +Traditional, old-fashioned men show too final, Home glassware 4.84 6698.16 1.65041448856828214 +Years must share new, white loans. Able Home glassware 1.64 1410.40 0.34752000469930625 +Single, roman facts may hear by a rights; different, able preferences must produce as internal surveys. Similar heads might stabilize direct na Home glassware 6.70 8825.39 2.17456010654651897 +Stones should send perhaps at the groups. Perhaps individual facts Home glassware 4.18 26041.20 6.41650449969907389 +More black members would run more central poor phases. Personal responsibiliti Home glassware 8.30 423.06 0.10424121751849724 +Safe, distinct millions must not deliver at the men. Indeed old claims might put exercises; particular, wooden households should learn clear, lucky votes. Mean, level terms might write bot Home glassware 9.86 7952.69 1.95952840766599957 +Significant difficulties could observe numbers. Very alone centuries affect forwards by a matters. Glad fields ought to spread hardly british str Home glassware 3.06 501.96 0.12368203457094708 +Novel, small attitudes may warn now however good terms. Aware earnings must eat much; lat Home glassware 2.84 5534.76 1.36375483636523840 +Cold, old days stem thereby difficult, nuclear men; likely contents shall threaten often outer years. All real or Home glassware 9.08 11902.21 2.93268298009935465 +Now strong fields may not feel. Again Home glassware 3.96 9805.52 2.41606236279008890 +Even sexual men can clear thereby always male members. Shoulders extract. Negotiations used to alter else Home glassware 3.47 1371.15 0.33784887581073012 +Conditions could not estimate following problems. Theories get sure; extremely complete scholars ought to thrive only strong, european businesses. Important, social p Home glassware 1.56 6751.07 1.66345141670827100 +Holes buy then markets. Practical themes ought to escape above australian children. Home glassware 1.43 3401.20 0.83804951785541719 +Willing, due values will chat hardly gmt central records. Necessary, adult stairs make fast in terms of a years. Views would not dig Home glassware 0.24 2373.76 0.58489016332602467 +Moments used to contract really boats. A Home glassware 68.94 1997.56 0.49219516490864023 +Insects indicate germans. Other, particular properties might Home glassware 4.52 2374.24 0.58500843445638178 +Persons might live here doctors. Chil Home glassware 2.86 15578.10 3.83841561628351009 +Materials make apart colonies. Rates make naturally poor, appropriate companies; c Home glassware 0.80 1956.16 0.48199427991533955 +Used groups ought to fail high from the districts. Immediate, main walls could exploit rights. Therefore late friends ought to try away. In short widespread lakes sh Home glassware 80.17 9287.91 2.28852419657312357 +Too only affairs put nonetheless big numbers. Rapid students appeal for the Home glassware 9.29 13621.22 3.35624392967263487 +Good windows say widely actions. Simple, imaginative findings see to a recommendations. Environmental, l Home glassware 4.66 12892.65 3.17672560166371999 +Japanese emotions speak disabled, new techniques. Experts should not tell only refugees. Years cannot afford well head quarters. Offices make conscious, primary stories Home glassware 7.31 4129.01 1.01738058324126665 +Full goods should find then. Only able years exploit completely mode Home glassware 2.13 3040.36 0.74913919560946025 +Sexual, due tensions take quite lucky circumstances. For ever formal districts respond ways. Poor relations should not come correctly in an facilities; important times could look away common Home glassware 42.90 1247.40 0.30735710001553787 +Bad boys might claim shortly italian, good lines. Times learn additional, sick cards; measures work sometimes pleasant male Home glassware 2.10 3225.77 0.79482388369177617 +Children want on a paintings. Over nice teachers must not sell. Richly accurate pp. hate as african, fiscal days. Claims eat part Home glassware 7.95 6793.78 1.67397508332817129 +Always sad weeks would not put close to a masses. Fresh, atomic sides will not help together previous Home glassware 0.83 6893.14 1.69845720731209292 +As other Home glassware 4.88 2352.12 0.57955810653242499 +Serious branches use. Rich, english bombs keep much vulnerable consequences. Little, furious sales can keep to a gentlemen. As gold customers overlap betwee Home glassware 2.54 3062.18 0.75451560407694385 +Really different shares ought to help clearly p Home glassware 2.82 6640.72 1.63626137663554805 +There possible newspapers experiment. Annual accounts might visit possible, prime groups; competitive, universal pr Home glassware 1.12 63.36 0.01561178920713843 +Recent, labour complaints must read in a units. Softly old courts rely even. Actual Home glassware 8.70 2861.55 0.70508073556955459 +Well new carers shall give together with a samples. Individual, central birds find there weapons. Kind details proceed ultimate miles. Unlike, independent months mus Home glassware 0.46 6486.44 1.59824706415326716 +Overseas businesses conceal gmt in a farmers. Level functions could support all right dreadful processes. Walls buy furth Home glassware 3.81 10274.91 2.53171920836992962 +Mental techniques might prohibit by a chiefs; other, waiting defendants vary else. Now old skills would see. Common jobs will no Home glassware 89.76 2200.15 0.54211297386498769 +Dogs will cover never. Bitter children restore cheaply upper, short views; other teams shall exist too high customs. Yards must not help now present, coming mines. However federal method Home glassware 3.22 2352.77 0.57971826535478358 +More than divine areas will control together from Home glassware 4.90 563.56 0.13886016296677611 +Surely national arguments address working, soviet effects. Again central parents say english rules; carefully military chang Home glassware 8.61 13637.98 3.36037356330760394 +Classical, attractive employers want only prices. Financial approaches used to hear considerable votes. Bo Home glassware 2.50 13555.23 3.33998411323041478 +Other patients see normal colleagues Home glassware 4.62 1970.54 0.48553748586228795 +Newspapers ought to pursue. Well rare criticisms used to tell so. Powerful, new matters touch. Home magic brothers can read now rather supreme rats. As evolu Home glassware 4.99 1537.58 0.37885692628017534 +Surely additional years work never remote, great bits; women deal in a judges. Far ethnic hands might help afterwards already dead awards. Rich, social experts target social children. National Home kids 0.50 361.08 0.11815869948988022 +Yet black costs must not judge here lively variables. Full, po Home kids 1.68 3938.44 1.28880289248621866 +Proud investors may not visit extremely. Alone, everyday houses move widely global countries. Only single gardens come further shadows. Scottish, wo Home kids 2.68 31.68 0.01036686496022877 +Total, new savings would make short, popular consultants. Short, other contracts might discuss for a Home kids 9.91 1600.56 0.52376229105883094 +Effective, free arrangements will build social, possible agreemen Home kids 4.30 2319.90 0.75915688198341950 +Enterprises shall not influence perhaps delighted, big police. Novels keep early temporary bacteria; rates will not cope men Home kids 3.57 6583.08 2.15422668504996302 +Agricultural sites will not provide skills. Again Home kids 0.55 5015.40 1.64122394323015739 +Conservatives tell effectively in a parties. Dir Home kids 6.35 8063.47 2.63866491795631001 +Too old Home kids 0.95 114.66 0.03752098283900982 +Following occasions see then only real lovers Home kids 5.63 6310.36 2.06498263795546836 +Permanent details would help also off a owners. External children used to listen like a years Home kids 30.73 6001.32 1.96385334668939829 +Farmers might not assume now to the tanks. For Home kids 3.80 11826.88 3.87019153601106270 +Local farmers skip also shoulders; things ought to seem so only applications. Foreign, voluntary voices may not find new Home kids 3.96 2251.62 0.73681314651989612 +Now close items become already against a groups. Authorities would work as well natural, dependent parties. Operators should not fall l Home kids 5.59 7257.25 2.37483998524685165 +Appropriate items take mediterranean centuries. High, very days see ways. Careful, technical minds remai Home kids 4.98 10259.21 3.35719206656024705 +Dire Home kids 4.41 1733.90 0.56739605917110697 +Short areas would not improve below to the measurements. Vo Home kids 0.36 18342.34 6.00229046195084044 +As beautiful children strike really natural services. Too assistant pow Home kids 3.30 2799.11 0.91597207635182954 +Even growing seats may build for a times. Obvious, different systems require american settlements. Evil yards support worldwide possible members. Courses could build also users. Alm Home kids 4.28 2619.47 0.85718723981598684 +Gold, young schools shall not go far human hands. Aware terms brush almost. Real years treat early. Edges cannot stop still british assessments. Very royal skills shall say already other Home kids 5.63 4448.98 1.45587041890020849 +Dogs hang perhaps chief, visual brothers. Minimum, small families shall work strong mountains. Small, defensive factors make by Home kids 5.44 2978.61 0.97471109972181264 +So dependent things distinguish again new subjects. Critical, firm centuries increase then institutions. Effects allo Home kids 1.59 10537.48 3.44825227844417572 +Turkish, old women must improve far from full, new changes. Days keep again exactly secondary visitors. Things used to make great, other notes. General, hig Home kids 1.38 355.77 0.11642107155620551 +Examinations reduce other, late things. Police should help very strong boxes. Annual, sole reports might benefit fortunate, total seats. Never rural shapes shall cease pictures. Physical periods wi Home kids 3.60 1189.98 0.38940536506859327 +Likely products ought to work other, considerable arrangements. Also other funds kill possible, royal patterns. Old, good files know palestinian colours. Northern Home kids 1.60 3252.96 1.06448854296167261 +Minds could not decide later avail Home kids 2.36 7178.10 2.34893918469122957 +Teams make never features. Now russian individuals may reproduce indeed other visual lakes. International legs drive also married views. Catholic populat Home kids 8.74 5328.40 1.74364909261625606 +Healthy, delighted conclusions may offer experienced condi Home kids 4.30 1952.10 0.63879915053227863 +Reasonable pictures could not try features. Unexpected politicians remember always. Serious buildings pay thereafter aged a offers. Large, material products go tomorrow interesting, individual re Home kids 44.54 107.20 0.03507979557249130 +Equal supplies could get easily still new years. Equivalent, national policemen shall appeal. Tables would Home kids 7.14 13784.20 4.51069886315610630 +Hours get skills; foreign, positive events disguise currently apparent prices; other programmes may sink honours. For instance var Home kids 7.04 2430.74 0.79542781986826031 +Apparently effective deals could stand Home kids 0.92 1924.93 0.62990812398652687 +Funny times go actually much old details. Military parameters tell so studies. Values u Home kids 4.41 1907.42 0.62417820588508729 +Levels contact in a sides. Companies must not count with an boxes; yet physical days happen never from a opera Home kids 8.77 13024.65 4.26214607652284354 +Questions seem strongly. Political years establish guilty centres. Necessary, pale eyes used to generate social, particular assets. Conditions help as firm directors. Persona Home kids 9.37 8639.50 2.82716318888562125 +Subsequent qualities say broadly good objectives. Odd workers ought to make commonly therefore intact times. Objectives will not hold just with the types. B Home kids 0.64 3035.53 0.99333742401272873 +Soon artificial notions think no longer lights; clearly late members could not trace good countries. Cultures can proceed away wealthy Home kids 2.38 3035.43 0.99330470032282902 +Appropriate, new ad Home kids 3.99 396.54 0.12976251992831810 +Ruthlessly empty times shall not focus to a lectures. Skills involve even; boring periods re Home kids 0.63 1007.86 0.32980898102323771 +Lists could play round, new roads. Soon national calculations think usually at first similar benefits. Skilfully specific practitioners will believe that is bars. More immediate Home kids 8.24 3098.01 1.01378318546206881 +Suggestions must see much large assessments. Disabled charges might claim besides wide, white passengers. Democratic, wide relationships test little years. Working, bri Home kids 0.50 934.46 0.30578979263684908 +Strong settlements should close here. Forms may seem quickly other unions. Places employ difficult banks. Women must not accept too areas. Vast possibilities know; never healthy subjects cancel most j Home kids 1.95 10592.00 3.46609323417749873 +English requests serve also intervals. More late cards might make only other companies. Tragic lights learn more royal, attractive studies. Businessmen ought to defend close po Home kids 1.59 17495.72 5.72524515852189842 +Goals help still human plates. Practical groups t Home kids 4.79 16455.90 5.38497768620671273 +Full, good fans might not pose of course parts. Daily Home kids 85.83 7041.80 2.30433679535792207 +Due years show just ashamed homes. Large, australian parties suit there automatic grounds. Sexual steps might not mean today technical molecules. Al Home kids 6.52 4853.82 1.58834900509020269 +Then dark tactics should not follow then. Ashamed, g Home kids 1.43 11882.09 3.88825828520469372 +Vocational, political styles run incorrectly indeed only hands. Complete, confident employers expect big owners. Inc times should stop more; consi Home kids 8.09 3606.10 1.18004898147351569 +Formal matters must admire much. Capable rules rise however. Harder only studies would show more. Old stones oppose common, secure police. Opinions come grey, appropriate systems. Eye Home kids 6.02 261.24 0.08548736749400772 +Soft, good estates must not join most likely, accused pieces. Coming, historical pictures arrange; best old loans cannot Home kids 6.24 6536.61 2.13901998635356684 +About american members provide certainly increased, special experienc Home kids 0.99 5029.15 1.64572345059136780 +Trying, ti Home kids 3.34 16043.89 5.25015281145090918 +New, other companies could take always political years. Important charges wait sure other aspects. Legal grounds may not worry to Home kids 6.49 5131.46 1.67920305772776318 +Windows recommend never internal cells. Mutual, other moments should not see levels. Necessary, national costs shall not walk light, high types; more digital days might continue. Home kids 2.75 8373.49 2.74011490138339726 +Fresh, f Home kids 1.45 4190.94 1.37143020948299155 +Quickly wrong facilities prepare as. Similar surveys look hopelessl Home kids 3.16 116.22 0.03803147240144533 +Remote, left figures used to feed on a records. Over economic depths must understand in particular at the ranks; degrees can think go Home lighting 2.60 5654.38 2.08346575200781715 +Lovely letters would require now questions; communities will add years. Emotional, traditional times make for a patterns. Perhap Home lighting 8.69 2656.29 0.97876146321981272 +Moving, powerful drugs use so blind honours. Efficient, other seconds look just rare, planned homes. German, specified sons reside further red weeks. Available lists undergo young, milit Home lighting 0.67 10412.96 3.83685665573012774 +Different men may not inform by now between a eyes. Members can cause new minds. Strong, chief rooms will carry high lessons; natural molecules accept here because of a talks. Eyes may disc Home lighting 0.56 7704.59 2.83890530849746709 +Incidentally immediate flames earn. Friends influence certain, potential men. Early, opening conventions should see per a agencies. Economic, senior practitioner Home lighting 1.62 616.89 0.22730506045863602 +Original others get industrial yar Home lighting 1.48 6297.95 2.32060157486013180 +So soviet years get. Good things must appreciate well real churches. Overseas, constant boxes complete for no months. Subjects may not suffer widel Home lighting 5.50 178.36 0.06572019417303299 +Important, toxic commun Home lighting 0.33 431.67 0.15905716650971714 +Prisoners give fundamental months. Opportunities grasp capital actions. British iss Home lighting 5.72 5860.48 2.15940728609091930 +Under way long interpretations might take yesterday. Little little shares get quickly families. Measures occur. Forward daily hands m Home lighting 2.56 2458.11 0.90573820642898698 +New, future communities should make yesterday particular, primary relations. Significant students mea Home lighting 83.07 7959.15 2.93270286752800800 +Opportunities drop cars. Officials change as for a inches. Other, american societies take straight leading, total posts. Agreements get Home lighting 65.24 13670.55 5.03717874216279499 +Vital problems may lig Home lighting 60.33 6799.66 2.50546633500003077 +Rather american gentlemen might generate rather in a studies. Enough current negotiations used to co-operate nearly rough main rivals. Dramatic, overall weeks used to provide too other, great meal Home lighting 7.69 3528.80 1.30025466022538018 +Also new colonies go unhappily eggs; typically modern centres would provide then Home lighting 0.51 5329.54 1.96377216670187391 +Prayers increase ever depths. International, official member Home lighting 7.88 4324.07 1.59328728424415089 +Sick, old-fashioned birds might think there imports. Grant Home lighting 7.01 5314.03 1.95805720700449927 +Common contracts will undergo for the goods. Generous, long laws shall not reach less traditional men. All pla Home lighting 3.29 973.56 0.35872702533694772 +Front shelves produce more at a principles; previously everyday birds avoid on a matters. Up Home lighting 18.01 4993.08 1.83979696748983826 +Problems should prevent finally in a effects. Now economic men sign. Royal, permanent teeth can start colonies. Geographical eyes wi Home lighting 9.41 5689.57 2.09643218861327258 +Essentially everyday lines sing s Home lighting 6.37 2165.33 0.79785774864708186 +Famous, attractive arms shall go publicly just democratic men. Importantly private ministers ought to write. Levels bring most true, adjacent days. Successful, particular constraints may pl Home lighting 3.16 2680.48 0.98767473691932868 +Just familiar police work virtually rare fruits; blind police might not succeed possible, stable churches. Senior communications light old, economic activities; almost direct characters ca Home lighting 2.42 14392.73 5.30327994101837339 +New kinds will go wholly great, occasional models; efforts may seem then too local homes. However religious co Home lighting 4.11 408.39 0.15047919992332890 +More possible newspapers Home lighting 9.78 3183.02 1.17284532662394854 +Of course high Home lighting 4.02 405.11 0.14927062043864877 +Further high men can give stairs. Controversial, great fingers hate sometimes generally ancient books. Other dogs woul Home lighting 6.69 1549.44 0.57092115754353125 +Visual, sensible rates know instead excellent honours. Other, inc christians fill plans. Girls may not make to a institutions. Days could build appropriate, small statements. Left, runnin Home lighting 1.12 8531.28 3.14351523965302125 +Propos Home lighting 1.14 5525.76 2.03607322355673225 +Significantly severe hundreds let right. Domestic, good approaches like of course later main records. General firms will preve Home lighting 17.01 2134.46 0.78648309965559538 +More great societies press. Years make still other, lively standards. Decisions may strike to Home lighting 0.43 2644.48 0.97440984013625407 +Unusual, fierce imports can press fine rural contents. Perhaps public Home lighting 4.21 7474.73 2.75420894253753570 +Middle-class years record also recent problems; certain, mild others can show. Matters will influence solely books. Loca Home lighting 6.43 2611.80 0.96236826161206301 +Able, double cells monitor quickly tomorrow direct men. Different weeks used to become n Home lighting 7.19 187.35 0.06903273367525079 +Legal conventions ought to work in accordance with a cases. Together left books may not come sure subsequent things. Short, real products deal excessive, im Home lighting 5.79 9924.55 3.65689253801286467 +International, final writers must learn political eyes. Immediate times reach also also wrong requests. Isolated years may not plan yesterday minutes. Long, old researc Home lighting 0.62 4542.45 1.67375362200770182 +Alone new conditions will recognise personal, hot others. Sooner scottish eyes permit probably only advanced cases. Never impossible services use again direct Home lighting 4.82 8731.18 3.21717226373459388 +Usually severe kinds cost incidentally conclusions; normally continuing concentrations ought to tell amer Home lighting 0.90 8588.69 3.16466906532847440 +Empty, splendid pounds make relatively skills; public, simple exchanges might exploit simply. Basically quiet perceptions shall not sleep. Old, alone individuals get permanent, new minerals. Fo Home lighting 2.10 4427.11 1.63125436659215111 +White, fair artists take. Simply silent years could create general, alternative issues. Deliberately natural moves take so n Home lighting 5.13 1353.00 0.49853903743055412 +Regular villages will raise finally small, rich terms; working-class, smooth states may violate european interests; discussions should not tell particularly times. Delightful, previous obje Home lighting 2.57 1509.56 0.55622659966272526 +Happy sorts should care. Definite, sensitive pages should happen else smooth clouds. Local, legal years might not represent easy unfair clothes. Poor, other powers change only fo Home lighting 8.25 6600.48 2.43207460885411963 +Plates shall think; new, economic pupils collect entirely. Really powerful books develop yet girls. Best unlik Home lighting 3.44 2151.42 0.79273233991784386 +Writers say. Spanish, local targets find able weapons. Figures would win most into the effects; as steady workers shall understand. Social point Home lighting 5.26 5754.60 2.12039375077447653 +Fiscal, occasional subjects ought to provide ill altogether royal stocks. Individual students save within a students. Home lighting 2.33 6565.32 2.41911922632931676 +Villages Home lighting 3.15 5303.78 1.95428039611487386 +Rich, logical Home lighting 7.93 2820.76 1.03936361805070942 +Residents a Home lighting 4.83 13929.25 5.13250176432338949 +Ridicu Home lighting 4.71 6980.98 2.57227719846411656 +Behind aware variables cannot bring into a contents. Different, electronic mistakes measure; however additional students should like. Interesting sales wo Home lighting 1.37 1624.72 0.59865953059436060 +Common feet cannot send at a engines. Orders should think prime, conservative cell Home lighting 2.52 2080.16 0.76647521367445784 +Emotional areas make then new targets. Difficulties should halt much. Military circumstances might mount very much white systems. Other holidays drag further through a Home lighting 6.63 10785.78 3.97422940069306875 +Yet young minutes could not walk here; enough pale others may not Home lighting 1.89 7242.84 2.66876458378678093 +Difficulties apply just initially high surroundings. Enough usual requirements assist repeatedly for a students. Directions make too through the flowers. More national historia Home lighting 9.68 372.50 0.13725483476931368 +Always top problems change almost expensive women. Supreme, industrial discussions Home lighting 4.16 1004.00 0.36994323250574748 +Discussions emerge so annual lessons. Good, early faces play really legislative products. Cold, private societies understand clearly ahead fat manufacturers. Abstract causes become so as executi Home lighting 9.11 4351.81 1.60350862415422005 +Approximately senior colours cannot encomp Home mattresses 4.73 2262.11 0.80877478687478841 +Facilities shall look just much quiet clients. Specific prices should explain on a ways. Aspects ought to establish ill high chains. Suitable, enormous areas c Home mattresses 0.21 4913.00 1.75655053375646430 +Sufficient, united companies earn either for no months. Comfortable, big tears come spiritual, old bir Home mattresses 6.95 6514.82 2.32925107843014222 +Complex, social miles cannot tie faces; probably future universities get objectives. Given settlements cannot g Home mattresses 4.30 100.50 0.03593188044830545 +Even widespread figures help also new, coloured trees. American, potential chapters may get actually years. Genes alter sides. Fingers develop par Home mattresses 4.87 NULL NULL +Dark companies stem in a offices. However multiple hours will preserve most long copies. Over mil Home mattresses 4.19 265.00 0.09474575441592979 +Early children shall not burst environmental Home mattresses 29.32 1972.12 0.70509432905186207 +Strong t Home mattresses 3.26 972.30 0.34762753591927748 +Also unknown books want very structural eyes. Well existing years could not buy much constant simple clients. Clouds find; ordinary, magic years prevent surely. Pensioners Home mattresses 0.47 5228.57 1.86937663836414340 +Central, new children agree strangely legitimate, full values. Underneath adequate rights avoid just rough museums; dead, local shareholders spare various forces. Small letters force finally women. Home mattresses 2.58 4991.57 1.78464175611291563 +Terms connect too all personal doctors. Current, new hours used to revive for the schools; practical, willing leaders discuss cases. Ago new structures must answer. More willing minutes claim more. F Home mattresses 5.91 5652.60 2.02098057136409324 +Physically useless findings continue other critics; perhaps young forms substitute coins; arms command Home mattresses 0.77 13274.08 4.74589707085813303 +Groups make in t Home mattresses 4.98 5572.29 1.99226724480883542 +Skills should spend twins. Certain, industrial homes will get to a rights. Decisions could buy politically so difficult differences. Running magistrates cannot respect thickl Home mattresses 7.20 4964.20 1.77485612857191941 +Here extra efforts ensure eyes; merely little periods will not loosen home past a boys. Just local aspects must reclaim. Standard qualities might not roll today. Military, national clothes must go wid Home mattresses 3.34 4129.43 1.47639985153876580 +Possible, rich d Home mattresses 4.63 10156.22 3.63116500344963929 +Japanese years escape so good objects. Tiny features see then proud heads; abroad full secrets might not re Home mattresses 0.95 2753.98 0.98463363300521627 +Past, interior years fetch accidents. Away internal feet would not organise so square collective rocks. M Home mattresses 6.31 3321.81 1.18765054519388575 +National, difficult pain Home mattresses 0.37 987.66 0.35311921436391401 +British differences discuss almost in the advantages; in particular international operations go then in a architects. Regional, fair costs commit merely political items. Then difficult travel Home mattresses 3.06 430.92 0.15406732261476401 +Never arab policies follow only. Valuable employees might shed. Recently relative costs order just with a areas; sessions may come somewh Home mattresses 6.84 7661.12 2.73908903423006793 +Perhaps blank models work certain Home mattresses 4.17 1990.47 0.71165502563122929 +Keys must not read political, italian farmers. Red, single years should play however at the dates. Authors disturb no longer for a purposes. Ever essential agencies will answer as fundame Home mattresses 42.14 5401.80 1.93131175926026233 +Payments forget. Doubts make respects. Considerable, available states should go here. Only public pages might differ. In Home mattresses 3.45 2289.13 0.81843527851372585 +Well able areas examine respectively previous services. Surprised computers ought to love british, sole appeals. Common, similar inhabitants finish from a seco Home mattresses 7.94 3465.86 1.23915290716979022 +Social councils used to determine yet at the boats. Persons ask alive months. Individual, considerable rooms note cases. Then only policies may look to a Home mattresses 4.91 4363.94 1.56024448122963257 +Films must ta Home mattresses 6.04 6064.00 2.16806888595546499 +Educational hopes appear more high others; black thoughts might close always in a officials; close years base top workers. Regulations ask over high widespread Home mattresses 3.52 15000.77 5.36324253007489455 +Vital arms generate slow, neat judges. Specially simi Home mattresses 4.42 10296.27 3.68123724083058633 +Closely blind winners might come similar, local crops. Very difficult evenings can stretch only ago naked hands. Sufficient, similar Home mattresses 6.05 13831.69 4.94526001470668627 +Natural beans look often bacteria. Square, small items must negotiate for the forces. Hence chief ha Home mattresses 6.40 161.10 0.05759826806191052 +Large, very materials like otherwise long, rough concepts. Sources give as local children. Rapid customers remove gently downwards short expressions. Behind national crimes confess n Home mattresses 7.74 1076.05 0.38472139260098583 +Growing, social objectiv Home mattresses 7.70 8.96 0.00320347909270464 +Ago new studies shall not apply of course small forces. Dead parts used to point on a students. Then other students should pay only Home mattresses 8.92 16657.18 5.95546070015825401 +Good, ethical interests stress now because of the eyes; patients used to give so hills. Social operations will pronounce basic ideas. British friends store too p Home mattresses 0.68 2433.04 0.86988758612880682 +Following, combined cells must ease of course continued changes. German te Home mattresses 5.91 785.92 0.28099088041723599 +Old words cannot force. Equal, capital problems would not produce; great, competitive things congratulate only times. Vice versa unemployed complaints will say previous gardens. Difficult, uncomfort Home mattresses 1.57 1412.84 0.50513430818491411 +Now comfortable grounds bowl much only double groups. Good talks must not support somewhat; used, linear Home mattresses 5.00 5416.79 1.93667115117986530 +Respectively excellent things speak reliable, historical movements. Masters respond. Cheap ideas should featu Home mattresses 3.37 5563.35 1.98907091633910557 +Prisoners ought to leave. Main items should not strengthen ago allowances. Ideas provide together between a patients. Regional, english conditions argue also in a minutes; ordinary trials become lon Home mattresses 36.96 6326.30 2.26184930626979851 +Cases move so very natural tories. Therefore political cells win there already eastern events. Extra questions encourage skilled efforts. Serious, physical clothes would Home mattresses 80.68 751.60 0.26872041139250123 +Individuals recognise. Really elegant relations should extend totally types; attitudes would relate muc Home mattresses 7.09 1139.56 0.40742819585742244 +Evidently super tales may risk just; others match maybe. Lovers describe anywhere Home mattresses 2.32 9619.86 3.43939959651179740 +Minimum words Home mattresses 6.86 4696.59 1.67917721785777990 +Other, extra notes alter so social ways. Different, preliminary parts take so diffic Home mattresses 3.40 10150.42 3.62909132278695101 +So social decisions fulfil again comparative times. Academic governments ought to arise then on a decades. Home mattresses 1.81 1346.52 0.48142284240051991 +Often presidential councillors used to take in the friends. Exact, rich visits used to want sophi Home mattresses 0.41 8719.30 3.11742134520308145 +Constant, domestic things might worry like a minutes. Literary, kind sales tell however emotional branches. Too specific troops may not bring most fair unknown owners. Issues look official Home mattresses 0.51 148.32 0.05302901998102153 +Even successful questions continue indian areas; good, good jobs get nice, famous interests. Labour, generous circumstances help good changes. Strict, vulnera Home mattresses 2.55 2079.26 0.74340021632779686 +Good, full governments consider elsewhere genuinely Home mattresses 0.33 11909.49 4.25801364059989293 +Days should conti Home mattresses 3.57 1697.42 0.60688052249316052 +Personal, national arts ought to rely still strategic, dead instruments. Finally federal spots remember. Laws Home mattresses 3.72 13796.99 4.93285368384543056 +New products should not see. Much separate subjects give at least existing implications. Similar corporations might turn years; local Home mattresses 3.84 1888.50 0.67519757439427698 +Other parties will add regional, special owners. Little administrative horses may indicate; Home mattresses 1.41 23082.32 8.25264838516945064 +Often local men ought to suppress trousers. Angry studies might cool seeming Home paint 0.70 4572.36 1.91646328201692969 +Worthy, rich types force both shy years. Tropical, personal views might work. Other eyes ought to administer neve Home paint 0.28 12758.19 5.34747978724238078 +Rural others come as through a estimates. Publications should worry really powerful Home paint 3.24 4960.42 2.07911511634744823 +Early, dangerous weeks live still to a changes. Vari Home paint 2.74 12614.97 5.28745042138963413 +Perhaps de Home paint 1.44 1475.69 0.61852209813740890 +Clinical, national residents might cry; objects ought to justify only relatives Home paint 7.77 2688.57 1.12688976505180184 +Equal forces tell together feet. Never new police cannot place hardly big, independent years. Then old choices ought to afford especially; parties accept Home paint 6.51 6336.50 2.65588658515520978 +Always huge concessions express directly ameri Home paint 4.52 9357.30 3.92202754569128769 +Average decisions shall see. Lovely, wide temperatures prepare in a regulations. Right arms ought to make now applic Home paint 57.27 3310.24 1.38745711507049343 +Vast, separate feet wear financially other, dangerous workers. Other, old genes spin for instance ordinary purposes. Events could focus anywhere under fresh countries Home paint 7.37 10616.13 4.44965473893533925 +Quickly far walls shall see gold, true patients. Above bad pensions will insist as round detailed degrees. Free, Home paint 0.70 809.31 0.33921495655834654 +Probably political hands may park easily. Little, suitable officials apply today men; women ought to take to the provi Home paint 6.55 2700.80 1.13201585878437474 +Special words should tell by a follower Home paint 1.68 592.00 0.24813143824065086 +Both usual effects include repeatedly low, possible practices. Professional, past countries retain yesterday ways. Equally old Home paint 0.84 1006.06 0.42168093708849528 +Capital areas judge almost active, numerous f Home paint 9.32 661.29 0.27717371417932434 +Pale, original yards agree social, little generations. Weeks used to include now oral shows; Home paint 2.40 5882.28 2.46550438603752661 +Appropriate, like v Home paint 4.82 372.76 0.15623897790301523 +Attitudes must build ge Home paint 45.77 9930.33 4.16220788024372040 +Very difficult parts ought to know else areas. Members could not comment of course male, popular girls. Primary, worried actions might send indirectly elsewhere hard children. New resou Home paint 3.98 770.04 0.32275529172775471 +Careful universities may find cultural methods; artificial, apparent sections ought to tell highly reforms. Medical, glorious studies shall not agree straight almost actual states. Enough n Home paint 4.20 103.50 0.04338108759781649 +Players shall mean with a rights. Occasionally popular enemies worry. In general basic patients get perhaps parts. Other varieties enjoy thousands; classes shall not spend as for the families. New f Home paint 2.13 5837.14 2.44658436387167698 +Students wi Home paint 2.79 4724.08 1.98005534588495595 +For example clear witnesses used to enjoy yet international, environmental computers. Ill Home paint 9.67 59.46 0.02492212046923835 +Opposite youngsters see altogether. Plans may not say to the problems. Popular, new lands might create cha Home paint 4.08 7043.01 2.95201385277582167 +Objects sell so yellow ru Home paint 1.47 1136.47 0.47634110746174406 +Only horses can forget meanwhile animals. Rich exception Home paint 67.74 386.10 0.16183031808228935 +Responsible, useless sources explore there. Serious, conventional fields could defend once again famous efforts. Officials call as notions. Big, ap Home paint 3.14 8952.05 3.75217067855104485 +Aware groups could finish services. Companies make also glad, top ways; t Home paint 3.27 1574.90 0.66010507108986663 +Ever insufficien Home paint 2.77 3898.21 1.63389941531095878 +Se Home paint 7.48 13291.94 5.57119626555479193 +Windows avoid. Always noble funds should lead nowhere able initiatives. Under new groups wait plans. High enterprises could know inadvertently different, main Home paint 8.31 804.05 0.33701027519830292 +Human honours tell requests. Effective, late crimes know on a courses. Adequate, typical men should not tend already in a nerves. Home paint 1.35 7526.60 3.15470622138865334 +Patterns might not maintain. Great, vast eyes say still different views. Easily national plants develop together with the cities. Able g Home paint 21.04 8770.96 3.67626844518787008 +Possible guests want only; organisations weigh however explicitly c Home paint 4.69 2761.50 1.15745771402290094 +Letters state on a chains. General, criminal cases shall look unknown months. Special, poor nights give as ever Home paint 7.47 3235.66 1.35619758354348711 +Also inc goods could not lay Home paint 2.41 2540.30 1.06474373743703612 +Additional companies visit. Grey opportunities may not look numbers. Entire, british models assist also great quarters. Little males show Home paint 51.57 13562.60 5.68464095318015436 +Communist, different demands die perhaps kinds; likely, public forests should make moral, nice faces. Efficient, central services can p Home paint 0.27 668.17 0.28005740386698596 +Effectively initial representatives amount dark areas; comprehensive, christian words will not want hearts. There judicial men explain r Home paint 4.54 5116.69 2.14461427150600652 +Reasons will look probably key students. Now very bones us Home paint 3.58 54.00 0.02263361092059991 +Features need stages; french cells come hours. Still small beliefs look scarcely electric, good producers. Churches receive for the seats; businesses get appropriate, high ways. Purpo Home paint 2.89 7559.52 3.16850434123135981 +Managers ought to express so new faces. Universities should not appear at a stories. Accidents dismiss only single times. Other, current companies could not meet effectively that is to say perfe Home paint 0.74 6272.75 2.62916635004061266 +There blue items see in a conditions; lives ask silent countries. Here necessary months may encourage free Home paint 7.02 4828.00 2.02361247267882156 +New dollars might end old relationships. Other, gentle groups Home paint 8.34 2369.97 0.99335146062026237 +International years collect respectively affairs. Exter Home paint 69.84 5908.06 2.47630983954739820 +Colleagues attach th Home paint 9.80 2499.83 1.04778110347487541 +Furthermore additional Home paint 8.18 1563.59 0.65536458702482987 +Months find there costly foreigners. White, particular changes used to share away in a subjects. Muscles make fully less gold fingers. Norm Home paint 4.97 14512.01 6.08257755584916844 +English persons last there golden units. Special services help far vital fingers. Very complicated birds sho Home paint 0.74 1043.89 0.43753703896120444 +Hands might contact enough growing things. Criteria used to make convincing forms. Particular organizations sha Home paint 48.89 8562.98 3.58909551186812250 +New, american owners might not appear. Parties move heavily also high variations. Unable, american terms might create indeed years. Nations absorb over normal experienc Home rugs 0.89 2701.48 0.99827241978850362 +Concepts shall allow cautiously there Home rugs 4.82 8082.19 2.98659526203801105 +Awards might mention better real, video-taped fires. Familiar patients must yield finally never net rules. Courses should attend; black ac Home rugs 0.79 120.11 0.04438400444970800 +Smoothly main organisations yield here pensioners; subtle, british rights say public books. Only, social pairs take up to the police. Important, other men could go mor Home rugs 6.67 21599.16 7.98149374365127852 +For example brief children must change almost. Fierce manufacturers ought to throw comfortably alone, subsequent loans; other boots switch. Very main men k Home rugs 7.88 1113.44 0.41144722266657961 +Forms carry here american negotiations. Partly subject drivers should tell only stiffly local orders. Quite clean forces will enhance intentionally full ministers; stories mus Home rugs 7.64 9195.42 3.39796488383093785 +Royal, comprehensive reports cost loyal, critical minutes. Exciting, short areas ought to pay for a appearances. Public, large institutions can Home rugs 4.30 2726.74 1.00760669630502701 +Of course institutional forces occur violently from a governments. Patient, western teams count Home rugs 1.97 500.94 0.18511134117922509 +Great images may not pay only, certain plans. Internationally new years command so in the days. Stairs tell teams; else unlike customers see elected, different numbe Home rugs 2.11 8294.23 3.06494997274915987 +Organizations understand also instead accurate settlements. Costs become co Home rugs 7.44 12898.01 4.76617544944116470 +Broad, political premises must not continue even. Short local levels stay in a germans. Encouraging, poor priorities i Home rugs 9.98 13098.17 4.84014016787138328 +Consumers must light now human schools; systems take Home rugs 37.18 2295.76 0.84834753189127999 +Hardly happy reforms may not try quickly along a pp.; sure sources use then now different Home rugs 3.58 2396.96 0.88574376243253759 +However magic things should not take for a firms. Estimates supply; able, doubtful children must maintain left, lacking banks; simple sons c Home rugs 1.73 113.88 0.04208184519800805 +Ideological members get sometimes modest abilities. Used, certain services would make all victorian, angry regulations. Even voluntary directions must sail however equations. Other, specific others ge Home rugs 8.46 4771.52 1.76321009834210907 +Turkish members shall know to a subjects. No doubt decisive millions might know virtually public industries. Good, artificial Home rugs 1.62 4557.68 1.68419023728536476 +Softly social men get only with a miles. Only afraid difficulties should emerge t Home rugs 4.09 5355.01 1.97882597342628292 +Others could withdraw buildings. Clothes know partly. Inner prese Home rugs 4.44 7946.40 2.93641705902222677 +Parallel dead relations check further international men. Types improve apart half way steady ways; back metres shall not support at leas Home rugs 1.00 9684.36 3.57864188937285967 +Good, alone centuries might not see gently subjective ships. Less ambitious Home rugs 6.42 3762.17 1.39022704204943760 +Also other republics could not prescribe almost permanent mental p Home rugs 3.56 1252.71 0.46291138301718183 +Coastal agencies encourage. Obviously other events understand local students. Western subjects cannot set in a e Home rugs 6.19 3558.04 1.31479529757921118 +Existing services make talks. Concerned, running Home rugs 30.02 2214.66 0.81837881354250538 +However major months execute either elements. Enough left provisions used to prove so away gastric police. Animals shall add faintly things. Well modern principles might pay suddenly other, soc Home rugs 1.32 16957.77 6.26637032001602569 +Mental horses could grab Home rugs 1.74 1044.31 0.38590175411601501 +Other, initial companies could know definitely mere funds. Italian years get already thereafte Home rugs 8.14 4357.37 1.61017008965967989 +Additional, interior police provide words. Different, long qualities answer really concerns; then other words state dry, political services. Awfully di Home rugs 9.78 7977.70 2.94798328447619281 +Firm, main skills can measure already electoral, white activities. Fairly disciplinary men protest there new changes. Strong, good reactions might prompt arbitrarily wild product Home rugs 6.42 9423.50 3.48224682317728204 +Origins used to play very on a matters. Long, important shows tackle more. Further vast fingers succeed only. Much dead values must rem Home rugs 4.71 7612.23 2.81293189736242391 +Possibly southern complaints would not produce to a years; months take. Services give; always professional days might develop quickly major urba Home rugs 36.03 10189.52 3.76531263858453641 +British stories ought to read furt Home rugs 2.05 1296.18 0.47897476386331293 +Better silent colleges protect never concessions. Certainly material words Home rugs 2.45 7108.50 2.62678957314752580 +Still global systems would find real forces. Facts get rivals. Ahead british features must not rest nearly. Flats will restrict always subsequent miles. Then new children can allay only ordi Home rugs 8.72 430.95 0.15924807857465376 +Possible Home rugs 0.41 9833.88 3.63389371141365844 +True Home rugs 55.56 1867.47 0.69008239771622846 +Difficult writings improve full charges. Western incidents run in a options. Parts happen possible, forw Home rugs 4.45 2413.98 0.89203312847811273 +Past losses will feel nowhere options. Political, free situations must produce selectively classes. Difficult ways believe sometimes enormous scientists. Interesting, simple rights ought to flush ago Home rugs 4.83 1972.51 0.72889761566142310 +Minds apply reluctantly dirty goods; therefore extended unions make secret, working men. Followin Home rugs 0.63 215.67 0.07969609724143306 +Possible, false publications produce toda Home rugs 62.90 1868.41 0.69042975400781722 +Wonderful, scottish unions go nearby for a teams. Gladly current systems cannot look so major, emotional p Home rugs 7.31 5243.52 1.93762730007603777 +Dead names spend as a schools. Polit Home rugs 1.98 718.90 0.26565365747144353 +Standard, foreign hospitals say later adult difficulties. Things ask very into a metals. Enough public persons will not give however ago sweet c Home rugs 0.57 5940.00 2.19499614046511968 +Single institutions place also local ideas; variations used to appear yesterday domestic, corresponding attempts. Unlike, possible amounts open locally. National, main cig Home rugs 7.07 11038.74 4.07912318107709347 +Also noble characteristics might sound about a miles. Again social funds would stretch en Home rugs 7.90 2544.16 0.94013827958345773 +International metres minimise originally small allowances. Eminently favorite lines compare just never bottom things. British poets take countries; individual, in Home rugs 1.63 3135.51 1.15865864451006522 +Colourful bones may adjust so. Pupils might catch so. Final, Home rugs 86.39 282.42 0.10436208922393251 +Able armies bring certain, pretty requirements. Dogs pay weeks. Simi Home rugs 46.20 4674.82 1.72747674366484020 +Foreign, absolute bills sh Home rugs 0.23 4232.41 1.56399387455656182 +Levels look only steep, cold results. Examples used to ensure together only expensi Home rugs 5.36 2875.57 1.06260354404668084 +African days incorporate economic, similar cells; vast, automatic stations ought to plan previously in a judges. Blank times would pay into the workers. Gradually ultima Home rugs 2.42 1831.70 0.67686438223736696 +Hands order. Pl Home rugs 91.05 5998.14 2.21648049662785404 +Magic facilities should not fight only likely months. Later present members absorb once more Home rugs 8.11 1193.91 0.44118313839439580 +As active accounts talk slowly. Big implications make as a children. Rounds should not check. Likely, military Home rugs 5.59 2607.00 0.96335941720413586 +Prime members must need so regulations. Only injuries might run adequately to a shares; inevitably orthodox poets think yesterday protests. Thinking, full changes could put more. Months Home rugs 9.27 2740.60 1.01272835396611229 +Clinical photographs look also popular, common men. Loose, concerned earnings must go maybe only able enquiries; black unions observe exactly by a Home rugs 24.08 2749.12 1.01587673226859761 +Directly green hours will maintain also Home tables 1.10 1433.48 0.74353680625971805 +Then legal services may bother circumstances; obvious, original years worry scottish, static areas; much fresh journals mean exactly routes. I Home tables 4.46 15267.45 7.91912758652365733 +Small motives shall use large, patient payments. Answers refer here odd, average officers. Always powerful sections might yield into a Home tables 4.41 5271.29 2.73418403568155059 +Odd, poor times could recycle suddenly eyes. Fa Home tables 0.33 2225.20 1.15419685052398680 +Perfect grants fight highly as great minutes. Severe, available millions like counties. Young legs cook however from a years. Early armed services reject yet with Home tables 4.31 7602.83 3.94353875654740364 +True, particular parties drop for a times. Too mad Home tables 56.61 2020.10 1.04781280682343418 +Usually complete artists will give from the weeks. Units swallow political minutes; books might not arrest continually lips. Modest, royal problems must behave consequently genera Home tables 4.25 4496.26 2.33218098648974514 +Particularly popular detectives avoid rather free, major relations. Financial servants may know also widely surprising children. Delegates cannot get. However separate thousands discuss alway Home tables 4.93 7737.75 4.01352088807387150 +Nuclear needs can want. Overwhelmingly clo Home tables 0.43 930.32 0.48255096799365244 +Enough bad rounds arrange later well black places. Courses reduce then in a experts. Also poor systems offer wonderful performances. Economic, unlikel Home tables 21.49 7678.11 3.98258600574183368 +Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Home tables 1.61 33.06 0.01714800821423827 +Federal, clear months please. New lips take slightly interesting Home tables 3.47 361.20 0.18735210426445445 +Roots should not lend overnight in a feet; fine children retire once usually evident forests. Sometimes novel effects might not go tons. Casualties involve more. Correct, perfect deleg Home tables 3.13 10251.08 5.31716890637669900 +Provincial, important tr Home tables 3.22 2399.31 1.24450658162444130 +Western, complex eyes can tell only regular acts. Perhaps high processes could put. Changes stay in the prisoners. Ages give now fascinating methods. British, quick words shall not expect new Home tables 4.27 9672.26 5.01693871537351095 +Now professional schools will not visit useful lists. Beautiful plans can recommen Home tables 2.52 408.50 0.21188630839432348 +Personal dimensions can dissolve final variations. Gradual sounds migh Home tables 1.19 5519.07 2.86270591938765946 +Hard sheets share so books. Permanent Home tables 31.00 443.40 0.22998871271001966 +Current degrees see in particular be Home tables 2.99 2250.99 1.16757395675039954 +Vast girls call benefits. Good, difficult makers deliver local things. High, formal hours play for a payments; well new men increase all equal newspapers. Top, total rights Home tables 2.62 10786.92 5.59510564931431049 +Just responsible poems ask only just joint patients. Solid, equal books prevent. Never universal fields must ignore black, main cameras Home tables 0.32 6835.22 3.54538441337343388 +Most official languages might not feel anywhere careful points; good, post-war prices refer originally ruling varieties. Increased lands would not get we Home tables 0.35 13164.59 6.82838770287595335 +Important, small girls should realise only high numbers. Previous, statutory products can give rather scientific methods. Isolated, living estates move now old trees; univ Home tables 2.85 3966.40 2.05734603088187185 +More german bags might not give always about a words. Recently new guests ought to Home tables 8.63 4805.11 2.49237948428065532 +Too labour operators tell more Home tables 3.43 9131.41 4.73640331783356027 +Families must not hear more great, english feelings. Proper faces justify extremely skills. Immediate discussions undertake often pa Home tables 0.18 2677.96 1.38904053470664016 +Experts should not offer; low easy cities flourish particularly integrated, decisive Home tables 9.66 3549.82 1.84126867873766800 +Simply different statements complete always social, international speakers. Early serious buildings shall overcome just by a husbands; complex, common criteria will work little, fair countr Home tables 2.23 2835.45 1.47072957928196943 +Only long brothers detect in a years; commitments can imagine near little great fields. Civil, soviet patients profit already just long arrangements. Often indi Home tables 8.94 690.05 0.35792447272338536 +Central houses increase actually essential payments. Minor organizations take subsequently careful players; good, molecular righ Home tables 7.94 13582.01 7.04490075758821408 +Women get also chairs. Full, integrated paintings sit Home tables 6.34 1123.11 0.58254989429803830 +Wild volunteers expand approximately sales. Specific, close versions must stress longer able powers. Far me Home tables 3.86 2363.26 1.22580767974533392 +Bold parties could revert newly equal plans. Also other products cry as at least lovely discussions. Manufacturing, french letters lay economically ready duties; serious, stron Home tables 1.02 2741.71 1.42210724746095625 +Areas ought to calculate slowly charges. Difficult, national participants might not write away bus Home tables 4.21 5457.26 2.83064547208814138 +Closely young offic Home tables 8.10 25.92 0.01344453638575487 +Wide, new changes reduce highly on a notes. Nurses re Home tables 0.25 1860.34 0.96494632792728456 +Critical, neighbouring feelings should achieve unusual, hungry types; po Home tables 5.93 619.20 0.32117503588192191 +A Home tables 4.83 2031.72 1.05384002568155423 +New situations seem. National missiles will cater departments. Women come astonishingly. Spanish mont Home tables 5.87 8171.71 4.23861313382858538 +Highly tory votes could no Home tables 8.80 3686.85 1.91234525361961205 +Slight, present techniques run writers. Schemes make. Grand boys could help fine, past re Home tables 1.51 332.04 0.17222700083048022 +Dead, big talks will rest old offers. Dead, competitive authorities occupy alone Home tables 0.38 2425.28 1.25797705268686622 +Almost working things shall not see import Home tables 3.78 3316.68 1.72034046836055031 +Police know more families. Atlantic birds might keep there far presen Home tables 40.62 0.00 0.00000000000000000 +Obviously elaborate members would not retu Home tables 3.94 610.39 0.31660534585265877 +Quiet levels must achieve. Local, national metres fill to a businessmen. Real, key boots could not determine at best. Young groups may know ever happy, magnetic difficulties Home tables 2.15 NULL NULL +Labour, middle children might produce useful signals. Surprising farmers kill on the costs. Trees return recent, single animals. Original governments read over there. Previous Home wallpaper 3.08 5699.40 1.39109945794862842 +Once again only measures shall destroy independent, normal prisons. Present, industrial ambitions can prevent as employers. Large, previous origins say inside Home wallpaper 3.32 262.60 0.06409494291632625 +Reports can say. Constant, other keys will analyse here white months. Dreams would not change to a neighbours; visual, financial wages set in a girls. Fingers Home wallpaper 4.24 9127.17 2.22774348871898495 +Nearer regular men like in a ministers; children come therefore female views. Only financial events must not allow old miles. Very british forces get. Home wallpaper 9.72 5545.66 1.35357487103333520 +Great, strategic live Home wallpaper 2.35 12111.89 2.95624866016307208 +Groups can consent close. Awful, soft friends pursue comfortable departments. C Home wallpaper 6.57 1777.90 0.43394668320996359 +Empty, additional russians should ensure commonly in a books. Sure, close difficulties follow always on a weeks. Royal y Home wallpaper 0.85 328.29 0.08012844177456491 +Educational, reasonable rooms mi Home wallpaper 2.73 737.08 0.17990518097778275 +Then french ministers aid Home wallpaper 3.16 7027.37 1.71522802361730232 +Old eyes would not develop to a parents; nice, red games come now to a molecules. Sheer centuries could follow as usually late powers; backs affect police. Almost tiny trees shall buy fro Home wallpaper 1.22 20810.71 5.07944123952101991 +American, long organisations terminate for a agents. Facilities determine open. Now general students rebuild even particular pounds. Good teachers might not press names. Guidelines evaluate clear Home wallpaper 4.09 293.44 0.07162231549644621 +Public Home wallpaper 0.64 1015.94 0.24796883589646797 +Initial unions agree still otherwise individual councillors. Leading minutes bring mathematical conditions. Full, huge banks must not feel exclusively special lines. Ago other cases will hold Home wallpaper 8.36 1699.28 0.41475725285169409 +Fresh, othe Home wallpaper 8.40 501.78 0.12247357371117359 +Ahead national cir Home wallpaper 14.29 13998.80 3.41680231110840781 +Still fortun Home wallpaper 4.83 4391.94 1.07197693675525478 +Minor, single things could cry too profits. Examples focus material, young observations. Existing tensions would stop away. Facilities reply most thoroughly small Home wallpaper 3.85 6735.50 1.64398891094027208 +Wooden, clear considerations will not become now proceedings. A bit institutional firms will Home wallpaper 4.94 9408.96 2.29652229284842735 +Thick, other ways come completely. Careful men would find later there valid children. Interesting owners allow a bit best wide polls. Miles behave other, considerable heads; inte Home wallpaper 0.96 3860.39 0.94223715416891351 +Marked, free flowers carry restrict Home wallpaper 0.67 4918.41 1.20047680193864503 +Less western books give physically only Home wallpaper 4.22 5084.28 1.24096205777082719 +NULL Home wallpaper NULL 15833.49 3.86461019693915650 +Liable, other others provide also in a resources. Months get briefly long sheets. Windows talk activities. American Home wallpaper 5.42 151.36 0.03694368073044608 +New citiz Home wallpaper 3.50 6508.22 1.58851481106966039 +Main elements write generally however secondary periods. Documents persuade empty, labour margins. Over other friends contend afterwards friendly, labour buildings. Canadian birds Home wallpaper 4.07 2883.10 0.70370194182048822 +Shortly economic records cause nevertheless by a requirements. Privately silent forms take. Pink leaves aba Home wallpaper 8.70 0.00 0.00000000000000000 +Stores visit values. Others cannot hang around rather civil brothers. Direct systems go then free, other instructions. Difficult, top feet will al Home wallpaper 13.91 2088.96 0.50986965710010998 +Small, social patterns design deeply without a judges. Moving feet arrange in the developments; sports say Home wallpaper 0.63 13980.62 3.41236496890650830 +Tests should allow finally times. Thus other differences act already important weapons. So ridiculous spor Home wallpaper 3.26 12082.76 2.94913866135441792 +Courts must not understand ideas. British figures would isolate sure young preparations; able, short governments should express more private properties. Countries de Home wallpaper 0.28 15297.35 3.73375009528203862 +Military, poor questions challenge that with a costs. Appropriate, main patients will not see concerned, industrial findings; terrible, concerned eyes decl Home wallpaper 3.37 3242.71 0.79147491372505823 +Green, european terms privatise new arms; also local duties need damp, successful professionals. Fresh, furious odds will undertake too only probable players. Home wallpaper 2.81 227.73 0.05558393507362900 +Impossible, other patients provide somewhat. Initially helpful ref Home wallpaper 2.44 10361.84 2.52909955562873563 +Always western women run soon in the solutions; left members should allow national, innocent products. Of course left tonnes thank unduly especially interested customers. Elderly pen Home wallpaper 0.99 7449.54 1.81827052952356833 +Artificial, careful years behave even specialist volumes. Assistant che Home wallpaper 7.43 6528.95 1.59357455275532468 +Short things come from a activities. Losses should not work ro Home wallpaper 9.19 3438.64 0.83929716111879700 +Courts can pu Home wallpaper 9.63 7132.45 1.74087576391298992 +Representative, keen problems might exam Home wallpaper 6.78 17424.37 4.25290936977512414 +Useful developments might control effective, unknown homes. Other, right marks cannot become by the moments. Natural, christian bars used to enable also new Home wallpaper 75.10 6730.56 1.64278316448937089 +Perhaps different figures hang new women. Dynamic goods finance now; birds keep already proposals. Schemes guess animal Home wallpaper 4.93 11316.14 2.76202340949412078 +S Home wallpaper 2.23 2663.69 0.65014873761153490 +Difficulties should Home wallpaper 3.85 3734.34 0.91147109341261905 +New, poor adults used to fear; new offers may make undoubtedly cells. Clinical dogs decide. Then poor models know then entirely rea Home wallpaper 0.20 10778.60 2.63082159831650459 +Significantly poor employees will not attend over interactions. Other babies used to choose departments. Young members repair. Easy patients find ever pers Home wallpaper 6.87 6138.42 1.49825468201232053 +Perfectly tall bodies think there a Home wallpaper 6.25 2518.24 0.61464755921404955 +Areas would stop also logical, local initiatives. Existing, increasing words should take open concerns. Objectives protect jointly at t Home wallpaper 6.48 7065.22 1.72446638458220312 +Human, back b Home wallpaper 4.28 8161.86 1.99213233355310951 +Measures should make rec Home wallpaper 2.45 3471.50 0.84731757172135024 +Familiar thanks should see proposals; more single lakes shall not announce employees. Specified lawyers canno Home wallpaper 7.89 509.65 0.12439446937283196 +Basic moves mig Home wallpaper 0.30 11860.26 2.89483125541807904 +Components could identify hopef Home wallpaper 1.39 1204.56 0.29400687143674770 +Social dealers shall emerge even figures. Clear prayers could not send. Home wallpaper 6.93 6706.36 1.63687647134932864 +Actual, urban police learn quickly low, british years; ethnic, common months should fail then overall markets. Years get. Criminal statio Home wallpaper 7.74 1379.50 0.33670591680530107 +Particularly tight problems cannot lead special, simple sales. Warm bodies get. New, primary attempts wo Home wallpaper 5.23 15517.89 3.78757910788967986 +Chief, other others speak fairly; established years may reduce political museums. Vulnerable, male features sug Home wallpaper 4.79 7653.42 1.86803319883727966 +Much following charges cannot complete difficult, effective jews. Poor, commercial pro Home wallpaper 1.85 5730.05 1.39858045566525218 +Special, long-term cases may not like sharply favorite arms. Insufficient papers bring. Legal cheeks could not apply with a sales. Terms give. Judicial, natural sets see at the cells. Home wallpaper 2.40 15153.09 3.69853936997697683 +Sensitive, labour areas would not suffer general, successful seconds; golden, substantial methods pay then available beliefs. Afterwards round years will defeat Home wallpaper 1.96 4949.14 1.20797732591358298 +That positive banks ought to fall perhaps eligible, white proceedings. Voluntary, political bodies suggest united, unlikely women; soviet, long comm Home wallpaper 5.69 NULL NULL +Later recent years could take further; opening intervals weaken; personal years say often. Main pairs generalize articles; functions know quite other varieties. Pounds include to the hands. Claims h Home wallpaper 1.19 7033.67 1.71676571645954473 +Long potential cards make previous subjects. Continued, firm rounds might support. Royal, powerful vessels exist employees Home wallpaper 1.91 7286.37 1.77844428490949006 +Societies could make now below a lev Home wallpaper 6.61 5369.24 1.31051458988596934 +Boxes would not let further level groups. Different priests get chapters. Languages may stop still legs. Blocks must make good, important securities. Complete diffe Home wallpaper 4.83 1053.00 0.25701437506051615 +Protective, absolute fingers could hear usually daily, rapid schemes. Normal y Home wallpaper 6.16 437.24 0.10672076481620141 +Brown, natural periods might avoid in a changes; standard, military improvements should seem enough. Things commit easily from a hopes. General courts could close part Home wallpaper 2.54 1591.79 0.38852128402429154 +Times used to remember to the trains. Evidently chief tests will not look often apparent foreign holidays. Images will not meet earlier rows. Today happy months cannot like as against th Home wallpaper 5.03 5511.22 1.34516881682907673 +Proteins must remember above beneath available rights; good pl Home wallpaper 0.82 8210.81 2.00407996285910406 +No equal occasions care poor materials. Students could not operate briefly shares. Very determined effects go already heavy factors; full possibilities make certainly by the posi Sports archery 6.40 8728.20 2.57886262177629984 +Appointments will not go inc, temporary factors. Static, specific proport Sports archery 1.85 1021.30 0.30175665035404036 +Lives shall mean servants. Short inner balls shall take policies. Sports archery 0.82 20373.51 6.01962413938563079 +Eyes can go extremely hard numbers. Early, real others would discuss also. Good members Sports archery 4.61 3215.40 0.95003263835149453 +Days can establish routine members. Associations replace both simple, crucial areas. Parties transmit variables. Statistical foreigners should not play Sports archery 2.48 2613.03 0.77205442090925102 +Players will come just about senior matters; external hours may become natural principles. Smooth, national sentences can support public women. Protests tell too in a leaders. Labour studi Sports archery 1.36 426.80 0.12610372894458477 +Just silver assets move quite both statistical difficulties. Mainly national hours must prevent. Electronic Sports archery 9.78 10843.65 3.20390042260999677 +Entirely social buildings step all including the standards. Massive months read children; irish years come for a words. Sports archery 5.76 12915.10 3.81593783901641692 +Religious, subsequent views cannot meet around important min Sports archery 5.76 23175.78 6.84759203186346949 +Shares take. Consequences warn liberal, fresh workshops; illustrations ought to measure sports. White, universal organizations assist young provisions Sports archery 5.83 3736.19 1.10390696121243713 +Long, immediate cars Sports archery 0.47 7961.21 2.35224523877909490 +Holy days like new years. Excellent, standard standards regain more simply friendly others. Easily previous texts can Sports archery 1.24 2736.34 0.80848799826669420 +Low days go photographs; attacks may not tear probably similar, mathematical police. Likely, small name Sports archery 2.59 11492.70 3.39567086607645118 +Now public weapons used to specialise always limited Sports archery 6.16 609.03 0.17994600290328131 +Materials go furt Sports archery 3.67 48.41 0.01430337750282884 +Previously white patients should set sometimes level theoretical studies. Federal, european trends keep. Social, other hills can leave opportunities. Organisers lower experiences. Recent criteri Sports archery 2.18 4063.94 1.20074505203152723 +Scientific, elegant blues must eliminate. Basically local musicians might slow never now spiritual bedrooms. Wrong studies ought to impose relations. S Sports archery 1.70 4653.68 1.37499156821657742 +Constant, olympic languages could not bow other Sports archery 2.01 7616.46 2.25038427215855694 +Strong, essential rates could Sports archery 8.43 4002.55 1.18260656112265174 +Critical, secondary cars will extend social parts; together serious voices see personally a Sports archery 42.19 29.70 0.00877525948841183 +Women aim entirely reasonable, whole surfaces. Young drawings meet then sure, executive projects. Public, new offers used to sweep too light, old ar Sports archery 65.59 3949.47 1.16692337009083694 +Marginal, bright boats re-open subsequent figures. Most anxious positions produce nearly together with a causes. Invariably necessary hands must not le Sports archery 8.66 312.64 0.09237364062145029 +So blue parents will get at a conferences. Toxic methods Sports archery 1.14 2037.09 0.60188529802184673 +Differences give financial, current reasons. Working, legal memories cannot launch into a activities; small, difficult parties coul Sports archery 1.62 7284.54 2.15231409945169992 +Competitive holidays should not keep all democratic, o Sports archery 61.08 8753.34 2.58629056869679390 +Crude, silly estates walk. Specific eyes mus Sports archery 3.16 11104.29 3.28090997254466541 +Normally eastern men protect also only explicit quantities. Royal, modest miles build by a opportunities. Shoulders judge more slightl Sports archery 5.58 12487.62 3.68963319503977423 +Nowhere other groups say home chief members. Contemporary letters practi Sports archery 8.43 2359.96 0.69728152802263887 +Current children take additional workers; far waiting arguments like bad, little days. Comp Sports archery 2.50 7478.91 2.20974329765919510 +Aware families give men. All social winners pose confident, new increases; most glad years wonder in genera Sports archery 1.55 2973.81 0.87865166394727186 +Welcome, united enemies fall. Nationally late profits complete mili Sports archery 7.03 11118.64 3.28514987064765227 +French photographs shall not advise clearly for an demands. Important, statutory cases rate well times. Other, local doctors assess terms. Normally white considerati Sports archery 7.09 408.72 0.12076175279810376 +Designs would throw especially linear, horizontal characters. Fundament Sports archery 3.73 8691.82 2.56811366756120145 +Changes set shortly. Mental, different jobs need more with a solicitors. Other, federal pieces thank then to a chang Sports archery 1.50 15462.27 4.56853304814429410 +Other consequences may shape quite. Personal, particular lawyers take brown, large men. Skills would gather as busy fears. Days will Sports archery 3.96 12677.69 3.74579190113278554 +Political troops forget plates. Emotional lists must intervene virtually in the children. Ready, only Sports archery 7.31 402.50 0.11892397118133873 +Months could not change curiously public contexts. Confident hotels would motivate in a studies. Workers sing fully again due positions. Irrelevant hands might create otherwise here strategic po Sports archery 0.40 1385.73 0.40943233437296029 +In short major reasons ought to sell already professional local institutions; corporate, able jobs will insure so su Sports archery 9.22 989.95 0.29249387644960565 +Privileges face mostly solicitors. Different soldiers suggest home. Deep stations make right parents. Safe, central things would tackle just. As famil Sports archery 37.12 16530.14 4.88404942356147718 +Goods go only. Accountants may unite. Almost agricultural muscles go just regional police. Real samples used to build auditors; following women can believe. Very concerned tonnes would fit there Sports archery 7.66 2295.32 0.67818278144583953 +Young countries should restore increasingly others. Combined, large activities match in a cases. Positions can Sports archery 4.34 2791.69 0.82484189094964351 +Local, main troops cannot support never diffe Sports archery 3.65 463.60 0.13697677773830717 +Earlier controversial auditors s Sports archery 2.90 258.93 0.07650430772169947 +Old relationships in Sports archery 0.71 2104.62 0.62183793348489221 +Individual, grand relatives must provide much areas. Italian, respectable experts might revise nationally public standards. Comfortable forces record forward importan Sports archery 3.59 7433.10 2.19620812469070534 +Patient teachers shall stop already serious weeks Sports archery 2.66 11143.58 3.29251872491165869 +Schools will get financial, small years. Chronic, real societies Sports archery 93.67 840.45 0.24832211572510841 +More leading requirements cross; elderly, able structures know obviously only houses. Enough light populations postpone blank payment Sports archery 2.76 5506.32 1.62691538135460637 +Real pupils could adopt fine years. Big neighbours must understand for a visitors. Duties would not give almost at last blue priests. Previous, small miles make finally Sports archery 7.47 1309.14 0.38680280157102555 +Domestic, chief devices can capture provincial lin Sports archery 3.78 18988.01 5.61025976156763126 +Strings ought to include even. Difficult, medical Sports archery 64.26 5845.14 1.72702425071028634 +Big affa Sports archery 7.86 4365.75 1.28991882530417280 +There aware passengers allow all after a reservations. Simply environmental feet may close hardly labour members. Influential, old shareholders must Sports archery 2.48 5434.29 1.60563316112058941 +Bad publications improve by the years. Regular movements might give at least profits. Hard tests might not meet Sports archery 9.45 12999.48 3.84086903078854452 +Sources make visual representatives. European regions will not run unacceptable loans. Right, natural firms get merely moral buildings. Virtually various sa Sports athletic shoes 2.23 3212.86 1.46013319558188889 +Distinguished powers upset very at a makers; animals shall see afterwards complete, working institutions. Sports athletic shoes 4.30 909.15 0.41317707424639551 +Seriously social measures might give. Less technical travellers contradict entirely for a possibilities. Major, young police give only; more than important findings be Sports athletic shoes 35.35 15716.62 7.14265750276894310 +Priorities jump now important drawings. Both still movements will determine early massive, right patients. As huge goods might include at least chi Sports athletic shoes 1.75 11184.41 5.08292559090593238 +Degrees know as after a heads; new, complex ma Sports athletic shoes 1.41 3007.89 1.36698145504591167 +Real, comparative methods insta Sports athletic shoes 1.70 11493.02 5.22317810906375025 +Develop Sports athletic shoes 6.28 2742.72 1.24647090697582786 +However local things might not feel regional, responsible roots. Local, suitable nations set strong days. Patients might seem to a rooms. Sure othe Sports athletic shoes 2.00 303.48 0.13792111146928022 +Enormous, pure beaches lie highly financial periods. So united ships used to stay. Simply famous tons shall ensure separately extensive needs. In order educational statements must not pa Sports athletic shoes 3.52 3499.90 1.59058289848205428 +Grey problems must not acquire detailed times. Sports athletic shoes 16.36 1039.15 0.47225755563233998 +Current, political advantages will g Sports athletic shoes 3.15 125.13 0.05686723566017871 +Prices ought to go yesterday. Interests might rest here funds. Letters damage also rich agreements. Central, i Sports athletic shoes 1.72 128.63 0.05845786400518490 +Generally top practices can reduce most links. Earnings will tell as techniques. Flat direct measures would not go far material whole sentences. Simply defensive services evaluate nat Sports athletic shoes 6.06 794.64 0.36113625945020704 +Sentences will retire always from the marks. Modern activities may perform lon Sports athletic shoes 4.66 1180.16 0.53634169932643252 +Almost uncomfortable shares may believe wrongly constant levels. Red, other words used to resist more frien Sports athletic shoes 0.12 23738.70 10.78841402674246177 +Items used to thin Sports athletic shoes 4.26 23.25 0.01056631686325545 +Eyes may not give children. Good great beans shall cook. Visible, Sports athletic shoes 36.86 5204.23 2.36514164340902922 +Religious, alone results go all investigations. Banks ma Sports athletic shoes 1.04 3489.00 1.58562922735046355 +Homes cannot inform almost fresh hotels. Plans could kill today hi Sports athletic shoes 3.62 7136.25 3.24317757915727874 +Woods wear indeed from a numbers. Counties must not receive as greatly public windows. Above hostile groups breed of course usually true members. Sources introduce similarly words. Largel Sports athletic shoes 8.59 4113.45 1.86942004736164067 +Military, considerable sizes wash social consultants. Equal ways stand detailed writings. Tough, potential directions interpret then. Free wives would restore still. Better fresh men carry others. St Sports athletic shoes 8.09 4091.45 1.85942181205017314 +As usual religious variables may believe heavy, available sister Sports athletic shoes 6.51 590.67 0.26843898415566016 +Objectives shall get with a years. Huma Sports athletic shoes 6.42 6968.96 3.16715008891839681 +Existing theories wait supplies. Proper partners measure things. Areas must not thank a little. Hard white rules formulate then major, institutional differences. Sports athletic shoes 1.47 16050.71 7.29448979527840609 +Absolute companies might not raise in order powerful, recent waves. Major chil Sports athletic shoes 0.18 14627.31 6.64760397062645716 +NULL Sports athletic shoes 0.74 2201.76 1.00062338997167000 +Clean, large conditions would understand labour dates. Large clergy should give high jobs. Patients might escape. As national polic Sports athletic shoes 5.50 257.64 0.11708842480211334 +Particular, financial years shape then twice visual friends. Limited, future women ought to come casual scots. Relations concentrate charges. Shares shall establish in a plants. Then double Sports athletic shoes 4.22 164.92 0.07495040761669202 +Presumably yo Sports athletic shoes 4.44 163.80 0.07444140654629003 +In particular financial studies can gain less than huge, model consequences. Really other activities walk o Sports athletic shoes 47.58 1719.85 0.78161204547397384 +Now political women could Sports athletic shoes 8.57 57.80 0.02626809095467377 +Chronic lines shall take enough by the sales; international, welsh angles used to rule now front powers. Standard othe Sports athletic shoes 3.00 16781.46 7.62659027045362857 +Skills use rather than a principles. Easy employe Sports athletic shoes 6.29 9250.24 4.20391255488860762 +Accounts could think aspects. Industrial, large Sports athletic shoes 1.92 6322.30 2.87326559589505180 +Cells call no doubt pilots. Arms should pay rather good duties. Thus long s Sports athletic shoes 9.73 857.50 0.38970394452651834 +Friends cry easily sure varieties. Appropriate proposals provide recently between a books. New, considerable forces seem like the elections. Right big clothes fr Sports athletic shoes 9.64 2708.86 1.23108271390099647 +Words live only anxious countries. British, northern substances criticise most extra, Sports athletic shoes 3.18 2390.50 1.08639915963923277 +New rules continue wet cuts. German, following procedures shall see findings. As good charges cannot pay notably routine, short plates. Problems used to alleviate a Sports athletic shoes 30.73 3030.00 1.37702968153393653 +Supposedly parental instructions see. Broken, raw habits should not issue at all friendly beliefs. Certain constraints know Sports athletic shoes 0.59 5983.42 2.71925641487913747 +Also other measurements pay at least around the artists. Perfect, good cul Sports athletic shoes 2.83 4854.06 2.20600154981736633 +Democratic forests use on a communities. Potatoes could not include still easy movies. Direct leads could sh Sports athletic shoes 3.61 1739.94 0.79074225217430942 +Levels may not go involved issues. Miles will beat good institutions. Tiny, c Sports athletic shoes 9.51 9805.35 4.45619075505900481 +Never national communities could turn so. National, whole styles buy far really high leaders. Indeed beautiful others liv Sports athletic shoes 5.39 306.50 0.13929359649839985 +More than hot women govern only full polic Sports athletic shoes 1.64 3354.48 1.52449456307325393 +Notably international minutes write too national, important visits. Human, clean patients Sports athletic shoes 1.21 6716.71 3.05251123176759302 +Major missiles may reply british dogs. Other, c Sports baseball 1.15 12361.94 4.21788969172030922 +Also other adults ought to uphold usually in a hills; carefully good signs would ensure with an benefits. Continuous, nuclear days shall feel just in the politicia Sports baseball 0.75 3265.70 1.11425572088612417 +Therefore unexp Sports baseball 3.99 3063.58 1.04529244615007878 +Often unnec Sports baseball 6.08 2524.58 0.86138583085852692 +Eggs shall not encourage as. Economic classes act other girls. Technical features wash even. Social goods can monitor probably Sports baseball 2.18 3658.98 1.24844272211406762 +Managers shall put even. Physically additional guests help therefore high times; here specialist successes tend old plan Sports baseball 9.08 251.02 0.08564793797863701 +Dreams cannot need further at a securities. Modern societies devote once again for a businesses; ways used to say to a Sports baseball 1.06 4758.65 1.62364974927113782 +Fun activities cost white camps. Bare, solar databases go especially days. More subject sites deal certainly; partly equal occasions hear subs Sports baseball 6.89 1014.60 0.34618117230947778 +Most other delegates enhance natural, successful shows. American, similar times can derive easy, small departments. Artificial, other manager Sports baseball 4.91 1022.10 0.34874016973932312 +Fully silent bishops ought to seek only. Just new forms change immediately deeply raw cells. White corners shall lighten really reportedly glad games; teachers think at pre Sports baseball 3.06 14501.24 4.94781811860939439 +Winds owe urgently military managers. Internal conditions used to satisfy now as disable Sports baseball 7.10 7772.75 2.65205963637738361 +Organisations restore far. Far notes might not ask very places. Innocent requirements would not change to a children. Cer Sports baseball 1.20 8146.44 2.77956253631857102 +Also international soldiers use shortly decisive parties. Major, above advertisements expect about already loyal stairs. Lucky, small towns appear. Then english children corresp Sports baseball 1.92 3722.51 1.27011913634314422 +Guilty, oth Sports baseball 3.01 5530.46 1.88699105678166221 +Rather american exercises might remember times. Below western accidents give Sports baseball 0.71 7321.35 2.49804211106642533 +Later federal objectives Sports baseball 5.97 7447.00 2.54091384800776761 +Feet used to make import Sports baseball 2.92 798.30 0.27237968643273813 +Parents induce free deaths. Empty, red rec Sports baseball 39.45 15343.37 5.23515258602214870 +Symbols could enable too wrong problems. Real, old Sports baseball 0.29 5569.42 1.90028419543056548 +Elements shall arrange more. Coins would constitute however. Departments subscribe only in a children. And so on significant areas protect within Sports baseball 1.17 1171.52 0.39972222253498857 +Residents will happen here police. Owners might not match lines. Temporary, good symptoms used to achieve about in a issues. Troops can arrange. Even true comments shall not get ba Sports baseball 3.86 3886.24 1.32598375623495459 +Relevant numbers happen by the variables. Basic, italian fingers l Sports baseball 8.19 5295.33 1.80676478135772420 +Fascinating companies could tell partly about a Sports baseball 8.54 2203.05 0.75167990504277057 +Rig Sports baseball 4.47 7838.81 2.67459928573946137 +Easily natural relatives used to walk thorough, real rocks. Front implications tell either. Members achieve in a words. So black ages help far Sports baseball 90.17 13337.28 4.55067536548368992 +Teachers might not send unusual arrangements. Complex steps ought to hold all but statistical, recent pr Sports baseball 7.75 1162.44 0.39662412964658915 +Kids live so other goods. Colleagues ought to gain at first burning guidelines. Electronic, public figures give. Little leaves interfere. Stages could not determine yet environm Sports baseball 3.90 6580.60 2.24529846491203378 +Only solid days cannot cope ever suitable recordings. Inches go ever chro Sports baseball 9.36 11126.11 3.79622491922354013 +Cities ought to assess to the parties. Likely organs help domestic, passive stages. Therefore private obligati Sports baseball 1.03 7447.72 2.54115951176103277 +Hundreds would give seldom national, philosophical words. Obvious things li Sports baseball 2.21 83.50 0.02849017138561147 +Most local companies shall see already. Politicia Sports baseball 18.00 3997.41 1.36391492213840880 +Surprising applications could not explore. Tonight expensive layers meet then between a statements. Days de Sports baseball 0.95 4521.40 1.54270013057369686 +Offices obtain surprisingly according to the cups. Separate, only children work also social rates. Public conflicts force at least. Gradually australian storie Sports baseball 1.45 8302.97 2.83297051867772986 +Conscious, solar ambitions support outside countries; warm facilities rise occupations. Appropriate columns grow. Availabl Sports baseball 3.35 2187.71 0.74644590229959357 +Certain places kn Sports baseball 4.63 546.48 0.18645878872825095 +Single, wonderful departments will appea Sports baseball 3.19 5797.68 1.97816642920876516 +Statutory Sports baseball 4.72 3059.64 1.04394811950026670 +No scottish accidents will rely chan Sports baseball 4.35 25561.00 8.72140444057023607 +Properly common methods remember thus successful levels. Statistical families exist; trees will not go simply. Bottom, full things could see in the feet. Used, de Sports baseball 2.57 12848.83 4.38401639286929566 +Good effe Sports baseball 9.77 8394.54 2.86421417129785492 +Central standards ease written eyes. Simple, brief groups send in the ideas. Technical, possible islands try on a parties; activities must change adul Sports baseball 5.06 9693.92 3.30756218201684687 +Legal, other houses compete well new companies. Young, able layers would find orders. Rather good beaches die finally historical applications. Comments Sports baseball 89.48 2008.38 0.68525856775370489 +Clubs may take major changes. Procedures need. Lawyers shall not say pretty Sports baseball 1.61 8727.74 2.97790189711445061 +Clear practices might not own as. External Sports baseball 1.32 525.24 0.17921170800692895 +As simple views visit only japanese, direct differences. Hours assist locally too severe products. Else lesser dangers telephone Sports baseball 7.20 316.92 0.10813299539554474 +Anxious, just years must come various schools; rarely surprising students ought to talk complex hundreds. Thin, other makers shall look actually american, ta Sports baseball 7.88 11407.21 3.89213614289414352 +Too particular pages used to give here by a markets; capital, different researchers gain specialist, small directors. Required patie Sports baseball 60.56 503.66 0.17184861940212062 +New friends would leave long motives. Dogs shall face occasionally increased schools. New, green parents decide also probably beautiful men. Real tanks shall Sports baseball 0.54 928.53 0.31681411780457264 +Important, private results identify sh Sports baseball 1.25 4287.60 1.46292765069398475 +Other, significant materials could not mention economic, current races. Animals go straight living, young groups; masters may date. Top, able computers avoid less hours; questions recommend Sports baseball 0.56 225.54 0.07695417071030911 +Only warm clouds ought to hold really Sports baseball 4.99 1216.60 0.41510350308664564 +Books change slightly. Radical, distinguished characteristics imagine always as a ministers. Red strings deal late, sexual states. Peculiar, strong patterns live always. N Sports baseball 1.51 2123.42 0.72451017633095930 +Real, social cigarettes wou Sports baseball 0.29 5316.32 1.81392656216471802 +At least middle departments arrange international, environmental sites. More key kids might take up to the relations. Policie Sports baseball 4.87 2378.20 0.81144102502109211 +Young workers ac Sports basketball 7.78 1526.51 0.57071382054190651 +Inter Sports basketball 85.58 1184.67 0.44291065357015702 +Levels evaluate old arms. Attractive, dangerous men isolate very poor things; solid, sorry others shall leave now Sports basketball 1.44 153.89 0.05753460497683867 +Others ought to ensure still buildings; new patients keep notably in a drivers. Relative, good im Sports basketball 1.20 625.50 0.23385467160317491 +Favorite, pure features see green decisions. Imp Sports basketball 8.03 5094.18 1.90455282332128144 +Also federal cells shou Sports basketball 6.62 8298.39 3.10250562475630792 +Considerable ears cross during a members; very southern politicians allow numbers. Patients deprive earlier shares. Men used to press beautiful tactics. Eyes might develop on a co Sports basketball 4.97 937.69 0.35057264111204009 +Youn Sports basketball 3.28 1166.47 0.43610624905668334 +Always front rumours ought to improve. Hours use about a centuries. As uncomfortable links learn neither about real reasons. Dark days could deal much big, sole Sports basketball 6.68 10473.18 3.91559083859462726 +About national assets raise much. Other inhabitants may like thick annual characteri Sports basketball 6.72 1181.36 0.44167314923281648 +Early types tell links. Local reasons succeed probably properties. Friends carry low fruits. Able, old tensions get. Recently other vegetables Sports basketball 3.00 11903.67 4.45040581730226223 +Cases should soften courses; complex letters use experimentally far whole parties. Great, liberal decisions confirm. Households know very reasonable users. New, short feature Sports basketball 2.58 5361.15 2.00436446469282357 +At all attractive answers would beat. Trousers might take of course fine constant lives. Ladies shall not challen Sports basketball 8.87 19675.51 7.35605104664266008 +Whole councils would see again white Sports basketball 4.23 4485.02 1.67680716104503839 +So early systems would place only to a m Sports basketball 2.69 249.12 0.09313809079101988 +Different plans may make so in a trials. Provincial, perfect items must wear together. Simple aspects must not prefer then in the sections; alone, good rights can put psycho Sports basketball 4.46 9055.60 3.38560250067100027 +S Sports basketball 1.06 458.00 0.17123171797642543 +Often final groups participate with the characters. Superior, in Sports basketball 62.36 9883.09 3.69497484632233713 +Decisions bring young farmers; easy other minerals credit preliminary, basic offices. Sports basketball 0.22 9644.13 3.60563525827070695 +Properly large others say briefly other police. Results used to prefer worried, old opportunities. Very big contents create forces. Possible, famous clu Sports basketball 4.35 9926.05 3.71103623192117389 +Succ Sports basketball 9.92 8445.05 3.15733716134675017 +Similar, new events may need sometimes combined prisons. Communications pay from a relat Sports basketball 20.67 3976.01 1.48650441701189361 +Charming, general guns would look superficially; big heads can set essentially straight voluntary refuge Sports basketball 0.21 5246.26 1.96141072653057135 +Authorities might destroy however to the profits. S Sports basketball 2.28 2179.53 0.81485734995886139 +Favourably major feelings used to turn new, necessary years. Labour products go pr Sports basketball 7.28 256.36 0.09584489786121490 +Different organizations shall split; emotional, com Sports basketball 2.22 12749.88 4.76677697902460058 +Smooth years help more british, curious arms. Inter alia acute members must improve also in a years. Now regional Sports basketball 3.91 2159.38 0.80732390210465840 +Women may not represent very common muscles. More late stones smile again on the surveys. Topics must not find as variations. Economic boots Sports basketball 60.56 202.95 0.07587658769282869 +Heavy paintings s Sports basketball 4.08 4622.30 1.72813181223238268 +Huge, helpful heads think low policies. Absolute tons restore generally. Tradit Sports basketball 5.01 24011.93 8.97730136644032550 +White interests might Sports basketball 53.99 3630.36 1.35727681151287298 +Outstanding friends must reduce figures. Travellers Sports basketball 0.95 3994.52 1.49342472072312426 +Redundant, new writers draw sharp Sports basketball 4.80 9195.80 3.43801884752753924 +Clear members work national, personal operations. He Sports basketball 4.17 4072.64 1.52263131855788049 +Times remove other effects. Almost english conservatives can measure however new, normal r Sports basketball 7.65 1107.60 0.41409661753425504 +Now due eyes keep about. Then annual progr Sports basketball 0.83 3016.20 1.12766180733732398 +Addresses retain once more applicable events. Following blocks follow for a develo Sports basketball 70.89 268.59 0.10041730814691726 +Other, g Sports basketball 0.70 15012.84 5.61282616791528113 +Political aspects ought to say months. Of course Sports basketball 3.77 123.24 0.04607553913409317 +Fortunately favorite decisio Sports basketball 2.86 9079.28 3.39445570390611328 +Ancient, similar ways equip immediately. Never european leader Sports basketball 0.67 5371.94 2.00839850451152582 +No doubt established kinds ensure both comparative buildings. Threats attract almost traditional students; questions must not fight widely clean, minor relations. National, famous assets go commer Sports basketball 9.10 1401.61 0.52401765989724377 +Only social changes could achieve again soon go Sports basketball 9.05 4303.38 1.60889770852705168 +Early favorite contexts will save quite as empty pages. Unusual languages suffer soon actual cars; corporate businesses ought Sports basketball 54.80 7564.49 2.82812362077617992 +Recently free woods know Sports basketball 2.84 3637.05 1.35977799097414435 +Confidential members cannot modify either dirty organisations. Men might think increasingly failures. Internationa Sports basketball 1.70 6383.10 2.38643925549196761 +Old, poor pp. form seconds; bags know much; Sports basketball 9.50 5416.98 2.02523753634047386 +Comparatively unable miles show already; interesting drugs will not run parts. Yet political priests will run strangely left, d Sports basketball 4.52 1863.76 0.69680093165009314 +However comprehensive times ought to level even for a blacks. New employers see; far presidenti Sports basketball 4.48 4373.10 1.63496381197097391 +Areas expect. Organic, democratic resources would last previously. Cheap, residential fields cannot achieve seriously about Sports basketball 0.77 2524.50 0.94383072495957642 +Automatically competitive deaths jump wooden friends. Average, legal events know. Losses ought to cross. Conventional toys st Sports camping 4.38 8168.10 2.37504813353538829 +Only far tests take to a others. Appropriate comparisons will say fully musical personnel. Beautiful, administrative aspects get standards. Huge, sin Sports camping 1.74 11263.88 3.27521175920551774 +Cells give only serious walls; arrangemen Sports camping 0.18 151.45 0.04403729628970441 +Sorry eyes could shake obviously across a commentators; more other numbers may control schools. Children maintain. Powerful elements gather very then active opportun Sports camping 3.69 5210.19 1.51497313143383954 +A bit important Sports camping 3.97 2060.79 0.59921835471020101 +Straightforward deal Sports camping 4.48 14808.62 4.30592001704617007 +Whole services live since the wheels. Sports camping 2.26 8417.24 2.44749086709509087 +So-called, classical travellers contain capital, new paintings. Japanese stories Sports camping 6.17 18270.48 5.31252915889810863 +Financial, massive ideas might boil also leading companies. Even long Sports camping 9.92 4367.79 1.27002748340183563 +Groups should display of course possibly productive areas. Gro Sports camping 2.04 12234.96 3.55757384359644646 +However general jobs tell basic results. Issues lose critical husbands. Back, Sports camping 21.20 4822.68 1.40229638871199501 +Equal, different facts emphasise necessary inhabitants. Complex, active moves might put in a reports. Commercial groups can restrict curiously to a players; identical purposes cou Sports camping 8.94 13999.26 4.07058144903669396 +Always opposite skills take well in the prices. Colonial, weak issues shall deny more just respective funds; mental, creative patients would not play even in Sports camping 16.73 5674.31 1.64992585480113970 +Procedures find groups. Possible Sports camping 4.18 5862.76 1.70472168501437704 +Wild changes shall delay soon representatives; other countries die also legal, superb boys. Never video-taped sounds think substantially previous approa Sports camping 75.50 1678.45 0.48804489902577986 +Dear officers communicate much long interested relationships. Casualties position normal, good issues. Aspirations remind now quick words. Financial, l Sports camping 3.38 1526.49 0.44385930943064297 +Exceptions say richly normal losses; british, old individuals used to win. Childr Sports camping 4.27 4862.61 1.41390688221379690 +Then bad merch Sports camping 0.84 409.38 0.11903590858421386 +More fine pressures free there into the records; rights turn seconds; great areas ought to drain allegedly especially gothic dealers; programs speak even european, o Sports camping 2.25 4430.31 1.28820649802073507 +National systems must believe old issues. Long police would make able towns. Slow years earn exactly nearer the terms. Social, old comparisons shall survive wildly previous children Sports camping 2.12 4781.18 1.39022938444641077 +Well main goods share probably traditional times. Enorm Sports camping 5.17 3862.11 1.12299030949772389 +Terms reduce standards. Free things put Sports camping 2.60 1759.84 0.51171076594568109 +Players must argue away significantly national sides. Elections might Sports camping 3.53 14678.84 4.26818373238141050 +Labour, bright taxes could not shock still with a reasons. Dominant weapons will cause home; women say therefore bloody, complete areas; dem Sports camping 30.04 3575.90 1.03976868803138980 +Unable school Sports camping 2.63 9178.29 2.66878227905467845 +Still royal companies reach years. Complex, british plants must tell however such as a detectives. Ite Sports camping 6.35 8374.50 2.43506330655747472 +Just capitalist exceptions communicate Sports camping 7.91 397.64 0.11562225484739558 +Available tests forgive also policies. Almost local rights used to argue there new only men. Chi Sports camping 2.78 316.16 0.09193021852065332 +Never top observations spend appropriate, common states. Homes make. There available hospitals will appreciate away upon a years. Roots hang Sports camping 2.07 4784.91 1.39131396097437774 +Residents will l Sports camping 7.50 7103.96 2.06562688247083863 +Bold campaigns get with a numbers. Public, medical emotions recognize sources. Very single countries shall fit enough along with Sports camping 4.72 5615.05 1.63269475425225965 +Democrats may say again. There private services can think about fa Sports camping 1.65 18235.67 5.30240741387437400 +Different, ltd. students may not try scottish sheets. Almost likely schools may not order. Partly effective c Sports camping 3.91 11958.94 3.47731518052689077 +Certain, official generations might allow polish letters. Months provide equally product Sports camping 8.26 3715.04 1.08022659100761608 +Central, clear fingers must Sports camping 5.58 104.95 0.03051643608850761 +Always clinical doors Sports camping 33.45 2954.82 0.85917651913334019 +Available implications try only; magistrates must reduce quite black, ugly girls. Animals read. Chief pupils will manipulate easy more real seconds. Men might throw only british policies. Aspects ex Sports camping 6.42 12904.54 3.75226841506993789 +Affectionately sad chains answer sideways small, concerned documents. Interested minutes notice as a yards. Difficult c Sports camping 0.18 7683.32 2.23408807744213704 +Crucial sources make to a police. Great farmers make recent limitations. Yet indian colleagues should get. Mea Sports camping 7.95 1656.32 0.48161013265475868 +Good, white statements de Sports camping 8.79 4572.10 1.32943494464283601 +Conventional workers shall not take numbers. French, premier things could remember as to a gardens. Red districts ought to implement flowers. Fiscal, curious terms study much explicit words. Third Sports camping 3.61 5559.40 1.61651333768889187 +Fresh, electoral doors get at a teachers; children become more ministers; comfortable places shall not lift much safe, genuine procedures; official, extra beliefs break. Openly new days find ther Sports camping 1.27 4702.53 1.36736023057922522 +Much basic birds can light apparently. Normal, close teeth cannot expect as civil ends. Long principal conditions could not cover less more new officers. Efficient words get to a years. Real, able Sports camping 1.68 3665.26 1.06575200131265745 +Far specific clothes learn indeed times. Gastric, steady criteria imagine again in n Sports camping 50.85 6713.37 1.95205456449265676 +Grounds will take then by the boards. Historical candidates feel suitable numbers. Normally inevitable savings return systems. Psychological rooms would turn almost Sports camping 2.39 16931.42 4.92316909306983803 +Accounts listen firmly incredible trends. Votes must not exert away natural fears. Able terms reflect well golden others. British feet could not re Sports camping 8.64 12203.84 3.54852504425319390 +Labour patients shall Sports camping 2.75 7756.62 2.25540160545821715 +Powerful populations can produce honest lines; soviet, working-class feet w Sports camping 2.14 2940.02 0.85487310556392702 +Minutes can compete much mathematical areas; pages put for example more good passengers. Differences undertake to a parts. About conscious situations know light studies; mad, l Sports camping 1.46 2184.90 0.63530596674397594 +Visual, surprising parties earn resources. Particular, just situations can lose currently to a others. Social actors want loudly prime years. Fresh, other responsibilities obtain offices. Afraid t Sports camping 9.02 6215.95 1.80741916059417696 +Great explanations would not fill; sure, political powers let eventually horses. Continually public examples ask yet wrong, dependent officials. Early, g Sports camping 1.82 3966.35 1.15330029804337451 +Trustees could respond further precise surveys. Conditions would weigh. White areas secure particularly living costs. Strong, bare provisions can keep so useful, physical feet. Demanding, supreme Sports camping 4.48 9027.65 2.62498050742654327 +Just available years undertake social units. Alone long-term years communicate very huge workers. Relevant, false farmers start hardly bottom windows. Associations shall Sports camping 7.57 5611.89 1.63177591730095251 +Steps would make repeatedly short pairs. As good stages protect skills. Plants could not sweep observations. C Sports fishing 8.71 4424.59 1.05964726402462346 +Christ Sports fishing 9.05 1582.84 0.37907514038334286 +Almost personal matters may deal; major, australian offences happen prime, usual hours. Functions might visit at the followers. Championships shall smile observations; compani Sports fishing 2.61 1554.46 0.37227840004061759 +Accidentally wrong communities look more goods. Rural matters recognize. Large, new days go hap Sports fishing 1.32 4303.95 1.03075513030558269 +Problems ought to remove rapidly then new authorities. Half way exotic months bar systems. Front, new models cause too; difficult, full others comprehend eve Sports fishing 2.89 2105.84 0.50432867101214193 +Delightful, married guns should go much tremendous, clear networks. Again just hours shall know there. Large, whole years cannot want Sports fishing 9.33 2187.51 0.52388786001109799 +Very modern weeks must prevent hotly to a situations. Points look strongly regulations. Times take good groups. As af Sports fishing 68.83 2026.90 0.48542329107363830 +Members support general, mysterious programmes. Front times want with the services. Now new details should impose never cheap live activiti Sports fishing 4.96 11202.69 2.68293781078382606 +Tests shall see famous, good words; sexual, significant theo Sports fishing 8.63 11684.99 2.79844407813042221 +Personal, lacking artists cut pieces. Prices make quickly for a rooms. High, overall types ought to use together supposed women; reductions shall give prices. Lengthy, fundamental meas Sports fishing 9.23 13101.80 3.13775661107533389 +Other offices shall embark blindly resources. Spectacular copies may look also old, other offices. Properties fill better important others. Very wrong supplies will not own both aspects. Certainly Sports fishing 7.25 386.95 0.09267084833042791 +Sheets identify in a persons. Successful studies cannot solve for instance impressive governments; public buildings can move to a women. Substances sweep even on a tales; however great spac Sports fishing 4.50 5339.33 1.27871880247087137 +Inherent, public girls run. Opposite, similar players might adjust though central ties. Entirely small generations achieve rats. At all western boxes prosecute almost suspicious, ordinary v Sports fishing 0.46 2861.92 0.68540264699268189 +Difficult skills can tell specifically significant applicants. Irish women find si Sports fishing 8.65 0.00 0.00000000000000000 +Usually english commentators will indicate still dangerous, spiritu Sports fishing 9.90 13087.32 3.13428878865945433 +Early, associated parents continue stories. Alive, key costs will not supply. For example excellent wi Sports fishing 0.65 9375.15 2.24525934545809862 +Just left grounds would not shoot other, accessible readers. Long, true winners shall vary; male conditions must hear never local, clean studies. Major, generous pp. must not get always gre Sports fishing 3.62 8.19 0.00196142718135729 +Groups deserve also only members. Inevitable, rare dreams worry much old enquiries. Please clear nerves turn necessar Sports fishing 2.58 3603.80 0.86307585789687587 +Foreign advances expand never new, colonial players. Colours confess lines. Urgent, massive items sit then men. Different countries cut however. Effectively old ideas suggest only actually particul Sports fishing 4.19 20.28 0.00485686730621806 +Sole, public skills require long opportunities. Parents destroy how Sports fishing 4.84 1396.88 0.33453948731311060 +Courses try military parents. Fast, w Sports fishing 1.64 6454.18 1.54571478453878082 +New parties strengthen please at all current things. Similar teams must lead most real firms. Simply tiny planes will set moving advantages. Concerned, average memories use Sports fishing 2.13 5552.34 1.32973267352104439 +International, new heads succeed altogether. Inc men see about accord Sports fishing 4.11 4917.54 1.17770410517847910 +Illegal campaigns ought to become here western, certain abilities. Indirect teachers would not tend no longer long, main agreements. Twice sweet patients ought to enjoy Sports fishing 0.33 2469.18 0.59134514867689882 +Common, preliminary children will not maintain early further international Sports fishing 3.67 4265.38 1.02151798178483168 +Northern, de Sports fishing 15.22 1489.04 0.35661093163959266 +Unable occasions command more effective, other birds. Proper songs know in a ports. Later wealthy details look now hours. Aware, black issues Sports fishing 0.59 4257.58 1.01964995589782473 +Points can appoint even pregnant ideas. Other, basic bodies shall frighten too modern laws; features accompa Sports fishing 1.97 15202.78 3.64092135826557149 +Home available features need with a questions. Hard waters can operate still more content bands. Organic, large ideas contribute points. Difficult, right produc Sports fishing 2.47 7589.73 1.81766821992220870 +Collective, full signals will assume only services. Political villages think children. So old Sports fishing 2.56 2552.33 0.61125878361338953 +Industrial, slight needs would disturb too for a folk. Now known buildings ought to suggest so. Papers create colours. Good levels tell into a r Sports fishing 2.72 5261.10 1.25998346078618504 +Norma Sports fishing 1.01 8662.39 2.07456009786539724 +Onwards horizontal sports find. Normal, powerful eyes come american, commercial situations. Major, enormo Sports fishing 1.89 13071.78 3.13056710631534049 +Shoes give more now annual ch Sports fishing 1.18 6235.99 1.49346035270723652 +As modern women may find only with a bones. However simple minutes end changes. Catholic hands provide hard able rights. Weeks used to affect then tiny c Sports fishing 2.55 3728.50 0.89294032303915358 +Strong, southern weeks use to a exceptions. Shoulders write natural, particular courses. Cold, labour things will hang. New authorities may bu Sports fishing 1.08 5888.16 1.41015837267164344 +Automatically private stands go always easy terms. Well distinctive depar Sports fishing 1.17 5365.88 1.28507727520164501 +Internatio Sports fishing 1.86 8437.51 2.02070347459999698 +Apparent, Sports fishing 7.13 2649.10 0.63443427913719237 +Special, easy things invest here hot Sports fishing 4.61 8905.67 2.13282334630014721 +Leaves could not help accounts; maximum, supreme expenses may not build in a officers; r Sports fishing 0.44 13341.40 3.19513853447621392 +Still original workers solve merely villages. Only long years punish already. Scottish features should not take from th Sports fishing 4.81 3.50 0.00083821674416978 +Settlements must make significa Sports fishing 7.42 7154.29 1.71338447732755427 +Shortly new terms would recover yet satisfactory, previou Sports fishing 2.86 3393.96 0.81282117172642234 +Public, certain lives could not choose indeed in a tools. Then bad things gain women. Sports fishing 2.62 392.55 0.09401199512109957 +Circumstances cannot take lines. Modern goods would make corresponding tools. Subsequently toxic practices see annually alm Sports fishing 3.56 12846.92 3.07671527285990692 +Also normal groups must not keep possibly others. Rates will not depend centuries. Fields could indicate already in a months; important arti Sports fishing 64.57 16106.48 3.85734892161020958 +Crops shall argue already for the responses. Easy committees like as with a figures. Easy current groups should not meet nevertheless; evident, international forces sen Sports fishing 6.00 1274.25 0.30517076750238473 +Elements take further vital, typical Sports fishing 1.73 6796.42 1.62767801268868558 +Good, silent examples close so literary, financial years. Often foreign interests discourage best suddenly whi Sports fishing 4.19 4776.06 1.14382098947415311 +Projects support indeed in a departments. Populations involve even with a terms; fine, classical miles visit continuously crucial, great days. Steady, sc Sports fishing 0.68 7528.93 1.80310719762348789 +Directions use none the less. Military, new recordings pass yellow tasks. Frequently wor Sports fishing 1.49 1880.44 0.45034751268760788 +Poor networks explain personally to a funds. Already federal words accelerate companie Sports fishing 2.01 7024.79 1.68237045779327228 +Sectors might not know properly. Large, electric workers used to drop even as ca Sports fishing 6.89 1774.46 0.42496630967414683 +Old others will cut other activities. Sharp passages avoid allegedly orthodox, additional firms. High officers must form. Sports fishing 0.25 2612.13 0.62558031541377612 +Very acids should depend much a little christian tons. Warm rules defeat at large details. Banks should not seek then. Times can back stiffly ordinary, chemical Sports fishing 6.07 10778.84 2.58142976306486528 +Factors might assist now absent, voluntary demands; political companies might know no longer concerned things; autonomous, possible events can dry at Sports fishing 6.68 6637.53 1.58962536740836076 +Pale, other animals assist in a words. Is Sports fishing 3.40 1226.34 0.29369677772719206 +Necessary women know little international troops. Immediate, possible drugs can try effectively too gentle spots. Northern, german ideas tell to a areas. False appropriat Sports fishing 2.18 505.79 0.12113189915246708 +Western, social things will go in order. Warm, male cards used to describe. High, briti Sports fishing 0.51 2346.30 0.56191655624158939 +Different, common buildings could not comply even. Impossible transactions build always qualities. Police move tiles. Options must use just different stages; words Sports fishing 8.87 4167.10 0.99798085560854416 +Members like very. Then interested principles could remember yet important, new agents. Necessarily due assets generate across. Areas give anyway as social projects. Main, Sports fishing 1.79 7991.56 1.91390268686784986 +Blind sessions hurt traditionally public, various clothes. High, southern schools might not tal Sports fishing 1.43 1122.60 0.26885203342999968 +Practical roads mean either dishes. Necessary issues determine simply fund Sports fishing 3.40 4810.52 1.15207383204675046 +Just formal teams ask still, certain interests. Well l Sports fishing 9.79 2218.77 0.53137433298902583 +Books must work major, able forces. Clearly future teachers would measure certain, direct measures. Hard tears go main nurses. Cruel patients used to leave much only days. Yet social defence Sports fishing 8.56 1810.80 0.43366939438361253 +Comprehensive, able acts must not resign. British, red forces convict perhaps; years want as well problems Sports fishing 54.91 119.66 0.02865743303067322 +New companies must benefit always to a companies; adults might get yet international, comfortable indicators. Dual bones shall find ever like parents. Wars need new, heavy purposes Sports fishing 3.43 7734.29 1.85228896636140409 +Backs think now back, british wines. Very fine shows get often serious, fatal prisoners. Good terms ought to come far easy, obvious shoulders. Machines play more ac Sports fishing 2.94 7583.99 1.81629354446177025 +Tiny values allow equations. Sports fishing 4.39 7729.84 1.85122323364381680 +Ill, simple objects shall bear solid trees. Ears should use there minimum, inappropriate personnel. Available practices should not apply increasingly pr Sports fishing 7.87 15557.69 3.72591893102937088 +Sure reliable suppliers show upright other women. Maybe Sports fishing 1.11 12392.70 2.96793389870653577 +Much common times achieve existing, continuing positions. Los Sports fishing 8.20 9965.46 2.38663298152977430 +Good, whole facilities maintain for a points. More worthwhile directors battle annual hours. Yes Sports fishing 8.90 603.00 0.14441277049553697 +Rules offer. Important, italian goo Sports fishing 4.06 3150.39 0.75448847104715544 +Vital, similar activities change thickly. Seats would sit essentially brilliant words. Hig Sports fishing 68.38 6302.32 1.50934575746174558 +Even useless times make old, old studies. Early public employees must open together warm consequences. Sufficient, evident men would operate stars. Various, other sections control l Sports fishing 89.62 2679.48 0.64171000047658609 +A Sports fitness 7.12 10468.61 4.22441966945963305 +Fast bizarre situations fulfil all as political plans. Thus labour conventions run more part-time experiments. Early considerable own Sports fitness 0.81 5713.17 2.30544721056249987 +Other, cultural differences might take. Musical branches take only new defences. Sports fitness 3.76 18567.33 7.49251276543379958 +Increased machines claim Sports fitness 1.76 2327.22 0.93910786084875139 +New parties survive Sports fitness 1.06 5055.94 2.04023384036732070 +Abruptly real years cope together; significant accounts provide at a others. Twice competent languages cannot impose most protests. Identical leaders Sports fitness 3.76 11311.78 4.56466578930728034 +Clinical, real figures figure effects. Full, pleased bacteria used to fit immediately more main things. Windows will not present perhaps Sports fitness 4.25 1715.39 0.69221484579083182 +Concerned clothes comment less small probl Sports fitness 0.73 1855.00 0.74855195549816254 +Large, working matters oppose etc far remote aspects; today amer Sports fitness 3.52 11563.15 4.66610164108818229 +Physical questions confirm much to the marks. Irish, pleased eyes would know in an subsi Sports fitness 2.86 8639.15 3.48617392255630775 +Little, national services will buy young molecules. In part video-taped activities join now Sports fitness 5.91 408.38 0.16479441918401058 +Intelligent trends used to bother open. Bedrooms will not hit all senior, economic boys; objects would sum. Often blue times should deal in a Sports fitness 3.84 1925.10 0.77683955230701493 +Absolutely wild standards impose only so scottish schools. New, complex incomes can establish children. Certainly free groups will rest. Impressive teeth must go front s Sports fitness 4.00 2927.91 1.18150552885316716 +Policies think races. Loc Sports fitness 40.32 1793.89 0.72389211183212873 +Shares could release barely months. Aware writings used to use so very impossible authorities. Sports fitness 6.66 3449.47 1.39197170562385268 +Boys might not fill in a problems. Military, young ways will encourage somehow inner, large matters. Ways will begin today whole firm Sports fitness 3.62 2731.00 1.10204603259594711 +Corporate heroes examine forth technical, formal shares; buildings may not emphasize abo Sports fitness 68.11 4428.60 1.78708204319092324 +Below old resources could cover lo Sports fitness 2.86 2908.84 1.17381017263141516 +Running children may continue common, small wives; great, subtle teams shall change bad, good lines; others may want; parties used to like near a sty Sports fitness 2.32 2591.76 1.04585822974766455 +Labour, dominant dreams come. Please various symptoms cannot persuade so owners. Primary colours would argue once small posts. Live, asia Sports fitness 48.03 4332.46 1.74828647176149287 +Deep, light measures could ask around experimental sections. Days attend social, wise cases. Children should find; as Sports fitness 3.91 12590.50 5.08067029417769025 +Times force also years. Emotional solutions ought to allow elderly differences. Too urban parents shall accommodate so. Traditional, effective im Sports fitness 3.60 8417.45 3.39671086674286159 +Principal eyes should pay frequently relevant areas. Light police m Sports fitness 3.17 451.78 0.18230771021830721 +Original hands know as. So prime things might not identify. Less than little journals let very hard things; nurses see; large bodies name once political, national c Sports fitness 6.83 1540.63 0.62169358447392677 +Methods develop warm males. Governments depend please with the hospitals. At random tory weaknesses enter approximately simply young me Sports fitness 6.01 24.98 0.01008023064600760 +Also new activities would not drop immediately fina Sports fitness 6.42 9171.55 3.70101438676505262 +Beings should affect close projects. In common labour metres might call directly Sports fitness 2.85 837.90 0.33811950593633983 +Men could not escape so old victims. Tiny horses give together effective teeth; little, beneficial bones used to forget again days. Of course Sports fitness 71.90 2421.19 0.97702776772646693 +Regions see in the cop Sports fitness 1.90 8595.06 3.46838219440648889 +Asleep, fat topics pick into a rul Sports fitness 2.70 3452.62 1.39324283158601937 +Conscious, central results play above about the hands. Stages stay so available universities. Tomorrow professional birds decide; enthusiastically big views appear new window Sports fitness 9.62 412.47 0.16644486527456987 +Please positive sys Sports fitness 0.31 4494.44 1.81365059346046449 +Simply necessary girls could not take supreme hospitals. Issues ought to Sports fitness 93.50 342.93 0.13838324641454710 +Overseas campaigns must finance just. Researchers believe sure, positive days. Workers appear from a values. Periods can lift ago related, extens Sports fitness 8.92 691.02 0.27884871821473869 +Regular, gold effects take gently for a terms. Good, strong difficulties attract articles. Ultimate farmers develop Sports fitness 1.12 3313.24 1.33699853425052940 +Round prisoners go at all into a lives. Streets find again places. Kindly liable men offer plainly on a contents. Early accurate regions should no Sports fitness 4.49 3281.89 1.32434780443658472 +More Sports fitness 0.82 1089.45 0.43962799348650845 +Solid, romantic feet would come so equations. Only economic feet will n Sports fitness 0.36 6592.06 2.66010749528906595 +Only subjects think for a goods. Windows wo Sports fitness 3.66 9334.78 3.76688292352837611 +Special miles must ease under across a conditions. Points might continue australian, australian places. Entirely Sports fitness 3.17 0.00 0.00000000000000000 +Men mean also weapons. Individual proposals ought to mean farmers. Sometimes valuable eyes might take rights. Rough, different rewards cost real, alone ministers. Requirements may no Sports fitness 64.89 3913.00 1.57902091744706739 +Together working cases used to buy in a structures. Millions must Sports fitness 1.88 3472.20 1.40114398915402693 +Sure, coming sessions could not pass very. Concerned children pick on a individuals. Easy pairs shall return. Reports consider subsequently rough sites. Vital, normal w Sports fitness 2.27 5967.84 2.40821471811329074 +Girls move ways. Other, human actors should participate serious families. New di Sports fitness 4.79 10717.00 4.32465299572712017 +Quick reasons could set only distant a Sports fitness 1.29 968.12 0.39066744968025936 +So close miles would seem american, emotional horses. Other, alive operations ought to want further red layers. Parameters might faint bad, significant stations. So prime newspapers wou Sports fitness 2.97 9281.14 3.74523746428690903 +Royal speeches take evil, front margins. For example hard events ought to go angles. Possible, foreign lakes shall not reconsider. Other honours hear momen Sports fitness 8.13 0.00 0.00000000000000000 +Points force into the symptoms. Local, strong negotiations get examples. For the time being fat result Sports fitness 5.61 19543.75 7.88652953114135530 +Subject, dead qualifications benefit more real nurses. Up to special writers give most responses; social circumstances de Sports fitness 2.69 12178.65 4.91447561877503891 +Just ready clothes try live skills. Girls investigate up Sports football 1.80 3028.92 1.26615656780156976 +Mostly furious applications cut in a workers; successful, substantia Sports football 3.20 4690.04 1.96054202463322710 +Dynamic, technical problems cannot go important, general sources. Overall inevitable subjects may take. Recent ends would n Sports football 2.51 10300.92 4.30601584472305176 +Allowances might lay at best children. Academic sections burst hot times. Short-term, warm goods Sports football 4.96 652.80 0.27288505720219244 +Sophisticated, unfair questions may remove separate premises. Typical patterns intervene typically walls. Naked areas ought to return now military, necessary children; young met Sports football 33.19 7921.58 3.31139830182558766 +Only available cars could not allow during a films. Cuts might not grow also unfortunately poor names. Windows go at first so key effects. Leading, possible relationships used to rec Sports football 1.80 5455.78 2.28063853765713464 +Pupils talk tonight even expected rights. However federal costs may not borrow large decisions. Social, american soldiers repair legal, economi Sports football 11.06 1681.47 0.70289221374658476 +British components must go. Wrong, overseas jobs explain with a towns. Quite ideological habits may Sports football 0.63 8173.32 3.41663127409899441 +Girls would face criminal, special offenders. Healthy principles get very greek, ade Sports football 1.47 435.76 0.18215746404170861 +Delicate readers gain too able officers. Feet see as international appearances; just prominent samples halt just. Substantia Sports football 94.83 12471.06 5.21318309049015641 +Daily, level areas fetch known, other Sports football 69.68 818.79 0.34227260414611390 +More reasonable opp Sports football 3.70 3418.34 1.42894287137950754 +Awful eyes get now like a gentlemen. Final countries may become french, turkish sciences. French lives repeat great, big standards. Large, able roads cl Sports football 6.18 5009.22 2.09396643112494858 +Thanks may add suddenly strong weeks. Times abandon as files. Systems feel cheap targets. Green, formal events understand french, rea Sports football 0.97 2280.64 0.95335872680393409 +Miserable officers introduce clearly. Much mathematical eyes could change so before prominent plans. Prices i Sports football 4.67 20055.07 8.38346955291662626 +Else social offenders will not support mines. Gently intelligent expressions speed days. Sometimes old houses offer really important, local month Sports football 2.19 15388.53 6.43275105592471583 +Critics can cover only str Sports football 1.79 10295.54 4.30376688392686948 +Sources negotiate never books. Sports football 12.71 1473.07 0.61577633457848288 +Young, previous metals keep here due, equal churches. Strong temperatures avoid. Established, average children could help also technical aspects. Feelings navigate now weekl Sports football 1.45 8988.48 3.75738645674136449 +White, vital departments should become aga Sports football 2.88 4166.35 1.74162784631488126 +Daily, marked years may not save players. Then hot families please universally always parental opportunities. Closely medic Sports football 3.21 1605.80 0.67126045474154508 +Popular, strong farms worry certainly followers. New documents will argue considerably under a men. Catholic, exist Sports football 0.10 1110.81 0.46434352081919024 +Clearly great options cannot believe. Responsible products ought to condemn at a systems. Dull types assure; real ser Sports football 3.03 8226.16 3.43871958050610814 +Succ Sports football 4.47 9246.93 3.86542435967320677 +Almost busy threats go together recent sides; still tired wines shall not admit on a Sports football 3.88 7510.88 3.13971648045159802 +Economic, crude hands put available payments; irish months pay main, tropical members. Neither soft syste Sports football 4.23 2877.00 1.20265059676885365 +International, profitable schools sit rather di Sports football 81.85 205.56 0.08592869540208744 +Young features may seem actually for the plans. Unduly Sports football 9.86 3012.65 1.25935534249415605 +Standards must pa Sports football 3.63 836.01 0.34947095078370849 +Very aspects use then. Popular, weste Sports football 6.30 1501.17 0.62752276550278069 +Models may register still digital, professional birds. There necessary things can fail never irish forces. All corporate readers identify more Sports football 68.59 9190.37 3.84178100974159524 +Again sexual officials shall not Sports football 7.81 11678.56 4.88190029662873252 +Ages must answer even such as a citizens. Fatal candidates say also. Thus great friends create normally Sports football 19.60 1325.80 0.55421416795138901 +Successive, joint Sports football 4.67 4363.92 1.82421654231892103 +Democrats take before. Joint years woul Sports football 65.80 7674.39 3.20806733171252094 +Hours take so. Now new things want common, recent drugs. Ships will st Sports football 3.32 1013.26 0.42356543054640551 +Quiet, small objectives should stay as matches. In particular formal students allow then. Professional, other demands drop Sports football 1.58 2487.00 1.03962184016827912 +Super stars might like approximately stories. Major practices might allow more fresh decisions. Advanced organisations wield. Towns must not protect quickly. Active, righ Sports football 4.05 6655.69 2.78222785902276383 +Cheaply financial tales allow unfortunately safe, red meals. Who Sports football 2.91 5952.36 2.48822012727947644 +Hard figures will not help twice central principles. Collective, impor Sports football 2.33 468.64 0.19590204229049551 +Advanced, foreign stories would greet always corporate games. Recent dev Sports football 3.00 634.63 0.26528958923441696 +Very questions make secret stocks. Aggressive, major years qualify for example senio Sports football 4.39 292.60 0.12231336969571310 +Matters reserve more proper, concerned birds. True months result together more chemical columns. Social views reduce in a affairs. Medieval, serious sports may n Sports football 0.16 7175.77 2.99963297628642230 +Proud things mus Sports football 28.70 17469.96 7.30283552990198210 +Unacceptable flowers should not give reasonable, ethnic governments. Employees shall complain Sports golf 8.39 4100.46 1.45417454300510042 +Crucial products would carry silently double groups. Really full systems run usual structures. Financial departments must meet well c Sports golf 1.50 12212.90 4.33114535351326216 +Different hours must not know towards a weapons. Facilities shall not know items. Today established fl Sports golf 5.73 437.77 0.15524940852766344 +Educational terms must apply automatic, other objectives. Indeed financial sources pass very unacceptabl Sports golf 6.99 16143.50 5.72508126771211978 +More black mothers shall repea Sports golf 14.90 7660.56 2.71671747490846200 +Admini Sports golf 9.35 2840.01 1.00717242550345943 +Separate, rapid bodies will start too religious surveys. Geographical, loyal things involve in order. Notes need dead for a members; at last economic managers look once more nervous skills; joint Sports golf 6.57 2341.31 0.83031498887521685 +European quantities would wait Sports golf 0.73 9236.58 3.27563236818065546 +Wet, suitable projects shall follow voluntarily all of a sudden resulting negotiations. High, video-taped services should not take all full eyes; wrong representatives follow royal, full figures. Fre Sports golf 3.35 7298.73 2.58839919478975935 +Good, interior faces contribute with a rights. Social, certain versions pick furiously between a troops. Forward political countries bec Sports golf 7.89 4757.12 1.68705042898124194 +Great, new errors w Sports golf 3.21 791.01 0.28052135742391451 +Stairs say long words. Newspapers will go exceedingly. Other, empty numbers must not provide therefore environmental months. Entirely bare groups buy. New days Sports golf 20.77 1505.63 0.53395199982069557 +Labour parties worry far well clear files. Finally domestic generations would not announce too; continuous, possible patterns might conceal Sports golf 4.32 2152.66 0.76341273216794201 +Live processes review home at pres Sports golf 2.74 4204.30 1.49100004174076658 +Judicial models should not pick. Close dogs can refuse exactly. European, r Sports golf 5.70 6536.36 2.31803463902021193 +Pages could watch fundamental, literary components. Financial, royal elements should overcome environmental trustees. Shared areas Sports golf 3.07 4544.08 1.61149857756900853 +Demands could treat lines. Conditions suck studies. Documents could not hide local things; gold calls see together. Preferences may refuse indeed in a pieces. Old, unknown boys emerge more opposite, Sports golf 2.87 625.67 0.22188568753798383 +New sources play just. English groups evaluate here indian changes. Familiar, able authorities get direct important, emotional orde Sports golf 6.52 7170.18 2.54281061753176740 +Most angry years help intimate conditions. By far urgent police would agree Sports golf 1.81 13747.41 4.87533926785135024 +Then growing levels light sometimes human, fellow cities. Users may derive odd championships. Stages support right Sports golf 8.86 5586.76 1.98127141098295675 +Brown customers can detect too. Then human numbers cannot prepare never victorian long accountants; interests share open in the years. Full-time, underlying Sports golf 92.44 6716.33 2.38185864718140065 +Secondary, normal Sports golf 6.04 7486.01 2.65481559890393074 +Wishes might behave environmental regions. Statements conflict now nuclear Sports golf 7.46 16077.73 5.70175679687386128 +Horses say. Other peasants can keep at first large kilometres. Necessarily new miles separate for an poems; interestingly indian teeth used to make further. Sports golf 3.40 752.00 0.26668697081299062 +Russians receive then definit Sports golf 8.76 20347.14 7.21584724907956645 +Independent, scientific subsidies might contain. Here certain instructions shall not imagine exhibitions. Either other attitudes buy finally. Public, right p Sports golf 4.05 198.74 0.07048054332363531 +Married professionals clarify plans. All basic children could prove more religious big trees. Sports golf 4.01 7501.44 2.66028764672260686 +Roles shall not remember primary, inc years. Young feelings can s Sports golf 5.74 3892.36 1.38037459802347363 +Particular, complete artists belong much enough active cheeks; profits may see able, complete processes. Here available officials take aside at a eyebrows. Sports golf 4.07 10080.46 3.57490338005521200 +Poles decide over for a managers. Properly other views include slim functions. Bright, other minutes should talk exactly certain weeks. Sports golf 6.56 1356.03 0.48089831520151552 +Inevitably dead trees establish original, primary events. Other women ought to issue almost long medical achievements. Catholic, hard cars need here difficult humans. Great, Sports golf 0.80 5928.82 2.10257851900994022 +Strong changes stay. Future claims will not recoup fo Sports golf 2.23 9989.59 3.54267752229221140 +Impressive records lie easy origins. Social schools shall bend else different details. Novel chemicals present primarily by a bags. Molecules shall see repeated Sports golf 3.63 4279.32 1.51760490417479657 +Also major pieces resign never. Substan Sports golf 4.63 55.04 0.01951921658716357 +Assets may not engage heavily always formal groups. Local, genetic offices cannot keep still sad, annual troops; supreme, natural gaps can see. Nearl Sports golf 7.20 4005.33 1.42043793192339857 +So overall investor Sports golf 2.54 15395.25 5.45972418538390139 +Brothers appoint even. Sports golf 3.65 3103.75 1.10070436922981335 +Closely substantial instructions wait for a companies; members may bring then characters; recent views should indicate near early days; objectives could not arrive categories. High gains speak Sports golf 7.73 77.67 0.02754465029660237 +Neighbours shall send important, excellent games. Plain important ways note monthly, japanese figures; routinely Sports golf 4.81 616.44 0.21861238868079779 +Certainly persistent players move often respective minutes; amer Sports golf 7.78 74.48 0.02641335849222279 +Impossible, natural cases may wait then products. Political sectors go here sure consultants. Me Sports golf 2.14 2979.66 1.05669747267637717 +Classical, small perceptions finance again ideas. Obligations determine. Clear, useful crowds could take thus formal, genetic individuals. Int Sports golf 0.68 14169.23 5.02493221735711581 +Forward working birds ought to try already then public pounds. Black, similar hands cover still at a rights. Right contracts save for example general, able feet. Systems could not t Sports golf 8.61 291.36 0.10332701571286296 +Young, severe parts must not act therefore rath Sports golf 2.17 1012.25 0.35898123165618319 +Only concerned times used to know however in the trees. Developers might not wear in the times. Studies see far variations. Calculations must not transport hardl Sports golf 0.15 8494.93 3.01261588958563618 +Sales include easier from the times. Significant, young features should not keep hardly social Sports golf 4.30 403.10 0.14295414618978261 +Likely, exciting negotiations disrupt even communications; all normal girls may think about years; allegedly old hands end darkly musical years. Individual, similar Sports golf 4.26 9885.12 3.50562860229110351 +Basic differences stem Sports golf 0.88 12915.95 4.58047284663835931 +Continental issues need famous areas. Thus christian years shall agree just foreign negotiations. Sensitive centres may not assess large remains. Men eat from the ideas. Other, specific plants say Sports guns 0.19 6159.12 2.82446517920513238 +Revolutionary son Sports guns 4.83 7287.25 3.34180595233776919 +Businesses may keep also behind a workers. Early, previous objectives hit wet, bottom requests. Under true hours touch similar, long sources. Widely able attitudes must appear now politica Sports guns 2.73 6762.87 3.10133441571052580 +Occasional, biological questions make usually for a tools; parts will use between a machines. Languages swim alive commitments. Other russians shall finish b Sports guns 4.12 2865.32 1.31398585630415545 +Again dull trials ensure suddenly; communities should produce terms. Too extra notes might choose properly social, absolute talks Sports guns 6.99 8342.32 3.82564268171208874 +Only other packages shall not lift procedures. Available, only types result obviously rough parts. Deep, back boundaries assert english, blue police; findings will declare restaurants. Little, daily s Sports guns 2.81 10686.60 4.90068866722739088 +Complicated, right projects forget naturally british, true weapons. Employers step also as continuous tickets. Ev Sports guns 5.02 8567.83 3.92905764075860015 +Then vague profits used to buy tonnes. I Sports guns 0.44 2445.30 1.12137199838780706 +NULL Sports guns 8.03 272.49 0.12495916895296837 +Vital, possible communications go yet operational effects; Sports guns 1.48 11987.62 5.49731378371310009 +Now good properties see quite mere exceptions; long publications ought to make alone facilities. Certa Sports guns 4.20 3874.40 1.77673237253249895 +Negative patients may not get for a eyes. Past little questions perform highly only, afraid acts. Again co Sports guns 1.13 5931.38 2.72002758099107309 +Differences imagine up a feet. Tender methods shall complet Sports guns 93.05 1128.12 0.51733618730677336 +Annual communications use enough in a standards; only famous conservatives used to kill new, public children. Men dance so examples. Christian patients shall cause as busy te Sports guns 2.43 22127.23 10.14716236203600213 +Courts define so. Appropriate tables surprise well to a agreemen Sports guns 7.17 131.70 0.06039532662154917 +Examples should not monitor firms. Fo Sports guns 3.84 535.99 0.24579568045470114 +New years can lend elements. Other, typical figures return under a flowers. Due, following others used to reject in full strong, lik Sports guns 0.78 4193.11 1.92288722862630256 +Other aspects might appear quite good Sports guns 0.21 5214.14 2.39111380676146088 +Strong chips meet to a connections; necessary, suprem Sports guns 2.74 4156.55 1.90612144926955361 +Artistic children can stay significant Sports guns 5.71 4613.16 2.11551484402024129 +Old ideas must withdraw holy pensioners. Additional bo Sports guns 7.83 1028.06 0.47145041371715901 +High, capital clothes can show. Prob Sports guns 28.98 231.55 0.10618479786803121 +Settlements relocate colleagues. Well Sports guns 0.15 9689.92 4.44362857506971716 +Major, late transactions ought to determine interested, industrial group Sports guns 3.27 2963.68 1.35909203949698443 +Films exclude british, young members; spots decide other, poor agents. Black, Sports guns 7.63 834.49 0.38268258247848571 +Badly heavy reports shall keep there important, given women. Vice versa pure plants reliev Sports guns 2.78 1558.80 0.71483853559355238 +Upwards new instructions help enough firms. Funds see then. Mines might play girls; odd difficulties bid complaints. Others go slightly at a fees. Empty awards find necessarily fi Sports guns 5.31 4316.40 1.97942587569669586 +Political, appointed actors might not take formal resources. Possibly new programmes might not use in a waves. Racial, suspicious reader Sports guns 1.08 15990.81 7.33310700754088619 +Golden, royal counties work then jobs. Patterns would take efficiently compl Sports guns 42.09 2480.64 1.13757830698921593 +Girls help diverse, clear workers. Classes improve no longer Sports guns 3.07 147.44 0.06761341653060903 +Social, large demands may attend subsequent, french sales. Small, able others will react in a principles. Enormous procedures could not move terms. Important members take so Sports guns 6.84 266.10 0.12202882622622805 +Wooden, english birds say so to a states; key, video-taped trends check largely ago fast ways. Urban patients promote and so on political minu Sports guns 7.33 4309.42 1.97622496924401239 +Alone, fortunate minutes can put particularly out of a consequences. Darling costs run already in a laws. Molecules discover. Temporary, political ty Sports guns 5.47 1876.47 0.86051646579755789 +Good definitions deliver a bit international childre Sports guns 4.27 10401.45 4.76992384273130321 +Suggestions go instead reasonable figures. More fat practices imagine Sports guns 1.92 7358.08 3.37428735692853857 +However old days hold perhaps new, gentle bones. Rules achieve also. Fine, vocational proble Sports guns 7.68 1967.40 0.90221538037384845 +Chips ought to finish. Bottles may not clear. Right, white wives used to accommodate about a words. Courts choose well new, future rewards. Permanent tourists serve ahead polit Sports guns 5.55 2717.44 1.24617066343555491 +Cold clients see lengthy, only spirits; numbers must not want once again tall leads; once naked lads make. Minutes lose front expenses. Probably alive p Sports guns 0.47 3757.58 1.72316075479575351 +Right, vital dreams vary most; documents Sports guns 4.18 2652.80 1.21652788505425697 +Directly essential organisations introduce onwards atomic words. Much famous steps ma Sports guns 62.90 380.00 0.17426138281084803 +Today keen pages wil Sports guns 8.17 1181.16 0.54165940768647699 +Possible roots must reveal at least upper, previous populations. So gr Sports guns 3.01 21554.07 9.88432116684688198 +Unusually global cattle shall tempt great prices. Worlds would not sign certainly deposits. Contributions predict als Sports guns 4.06 1782.00 0.81719416886560838 +In full possible products bear to a components. Lovely boards help alongside at the possibilities. True, dry papers should disagree into a c Sports guns 0.52 763.63 0.35018742041012600 +Resources go in a records. Permanent, flat applications would work Sports guns 7.43 571.34 0.26200657488197345 +Negative in Sports hockey 1.63 5985.40 2.60825063748619267 +Modern facilities see; certain procedures lure for a features. Still dependent companies put little persons; procedures find to a employers. Public boards know almost also tory considerations. Sports hockey 8.87 6280.74 2.73695059793581544 +Contracts will improve just by a services. Strange, educational passengers resist only english days. Difficulties should debate then impressive, linguistic applications; fine, new eyes build; roya Sports hockey 6.73 11482.83 5.00385916858448520 +Following parts treat perhaps appearances. Coming studies perform loudly so professional streets. Lesser, elderly years wear equ Sports hockey 2.07 8396.19 3.65879772779683831 +Girls would not enhance here inner authorities. Commercial others might not think normally problems. Loudly bright peasants see yellow candidates. Comfortable sessions may Sports hockey 5.75 3982.08 1.73526626433003944 +Depen Sports hockey 3.19 1800.84 0.78474990443589989 +Then sophisticated numbers might not facilitate alway Sports hockey 1.14 1035.30 0.45115144935834786 +Speakers get more with a Sports hockey 37.55 4112.16 1.79195107118074348 +Public, available symptoms take somewhat in a minutes; nerves seem. Curious, certain islands contact again vital respects; mass rules might recognise primary, Sports hockey 8.68 334.35 0.14569930174148904 +Foreign children increase about so tall leaders. Available, domestic telecommunications mess subsequently primary characteristics. Cities risk businesses. Elegant views cannot use f Sports hockey 7.88 2922.03 1.27332953691545754 +All british ways trap stages. Accidents welcom Sports hockey 3.21 4828.96 2.10431015444169561 +Much catholic guests invite highest problems. Long men must assume maps. Passive applications want independen Sports hockey 5.63 10772.75 4.69442845172910449 +Eyes must increase roughly. Services should love now senior, rapid sales. Sports hockey 0.88 9712.50 4.23240457055245201 +International places Sports hockey 7.18 5185.35 2.25961380076336237 +Reasonable laws shall pay significant boys. Widespread operations would not run then words. Substantial paintings make stil Sports hockey 0.88 10680.29 4.65413726752387621 +Military, special factors may adopt often young names. Actually large-scale workers make here advantages. Precious, odd customers study in the careers; usual women win then firms. S Sports hockey 3.48 7195.62 3.13562676715146818 +Parts work only windows. Positive, vital eyes could happen without a minds; common payments must not investigate only important seeds. Here different Sports hockey 8.94 1422.63 0.61993778267233306 +Colleagues come so; great places finish only large years. Regulations would know genuinely most other services. Opi Sports hockey 9.08 3086.02 1.34479126412522810 +Main months answer weapons. Little, norma Sports hockey 1.15 619.92 0.27014180091396407 +Workers ought to widen late, close benefits. Final eyes restore yesterday high, public funds. Quickly educational days go perhap Sports hockey 3.55 11162.51 4.86427370325224722 +Then suspicious authorities can advertise perhaps important massive mammals. Easy lawyers will put. Respectively responsible pounds might acknowledge ti Sports hockey 4.00 4553.02 1.98406410891291892 +Flights might work bits. Appropriate powers ought to lie just very parental pounds Sports hockey 3.03 1200.96 0.52334091048140775 +Little hearts must not get here. Best professional hospitals achieve there foreign shoulders. Women should not forestall certainly able deals. Projects sound years. Facilities shall find dry, Sports hockey 47.20 1750.77 0.76293096010153065 +As able participants arise. As red years must make often versus a models. Alone techni Sports hockey 0.13 10294.75 4.48613096038042269 +Small regions allow so new deaths; slowly late attacks would install automatically acc Sports hockey 5.69 12283.72 5.35286205110801192 +Interesting, complete times join secure reports. Ancient, traditional markets go lessons. Rapid terms figh Sports hockey 3.26 12950.49 5.64341962078700893 +Reports may develop relevant, clear cells. Intently inc Sports hockey 7.52 1084.78 0.47271329009460889 +Forces trust together from the systems. Reasons exploit even mar Sports hockey 3.36 2768.45 1.20640416302146057 +Annual priests look often practical genes. Needs may n Sports hockey 0.72 2604.48 1.13495115118789706 +Tenants shall not know so realistic years. Recommendations tell. Successful, proposed actions used to link also. Holes will not become only later previo Sports hockey 5.91 6583.03 2.86867915161739080 +Then royal plans would afford certain, terrible days. Priests ought to care rarely Sports hockey 4.15 6918.52 3.01487522980268214 +Complete clubs engage to a classes; other, small estates rob sl Sports hockey 8.86 2201.70 0.95943218975395970 +Details accompany ok. Black savings go ju Sports hockey 7.28 15049.92 6.55828573430617849 +Issues recognise only previous Sports hockey 75.67 4488.20 1.95581757462584454 +Very old efforts bring sorry supporters. Almost other subjects sha Sports hockey 1.96 7640.40 3.32944801862022696 +Too female dates will achieve also national, capable statements. Actual, small lights see then cheap effects. Free peasants used Sports hockey 3.59 8586.28 3.74163302095681932 +As national managers shall respect years. Other police could not consider. Therefore true bodies continue in the factors. Special relations would reach on Sports hockey 3.94 1856.04 0.80880434276737946 +Tonight certain authorities hang with a cattle. Internationa Sports hockey 0.61 9094.17 3.96295564204694903 +Psychological, ill activities talk rather right windows. Leaders would know adequately sacred, ordinary offenders; important minutes could affect again norma Sports hockey 7.66 794.92 0.34640134272571996 +British observations speak great quantities. Personal, ready th Sports hockey 1.66 274.86 0.11977541521359557 +Late, chief standards guarantee publicly police. Also political years might come curious years. Systems may not follow so with a times. Central, silent towns must apologis Sports hockey 40.41 5501.55 2.39740389859694645 +Columns blame rapidly. English users may not get excellent, female manufactu Sports optics 0.25 1588.38 0.64760161773605996 +Softly old women ask perhaps as a questions; relevant needs used to fall. Entries would not call together questions. N Sports optics 3.85 6270.40 2.55651744787279515 +Projects mount in general perhaps busy things. Accounts will fail. Often d Sports optics 56.35 1751.04 0.71392005484868258 +Good duties cannot determine gifts. Today social others succeed really quick eggs. Asleep, liable observers understand more after a operations. States must wish just similar women. Questio Sports optics 4.66 2203.00 0.89818957923956490 +Solid police must lift increasingly western girls. However central days choose widely over a drivers. Able years release commonly christian, aware muscles; sometimes important Sports optics 2.47 24705.19 10.07260291018316218 +Mad, social circles could arrive increased eggs. Shareholders search very low carers. Fast, significant patients will not seize then capital memorie Sports optics 1.38 6498.54 2.64953286803063189 +Obvious eyes talk lives. Neutral, real guests must stay in a departments. Hands can drop in the rounds. Flexible, mutual margins may pass like women; large clubs try. Old, sure records would Sports optics 6.07 1813.00 0.73918189158480761 +Circumstances join by a members. Human, personal priests will not obtain again wide, statutory days. Whole, new kids shall not encourage Sports optics 4.53 6033.35 2.45986931362007665 +Nurses should see certainly eyes. Clubs shall go individual procedures. New, internal police might read too international children; healthy, sufficient years break well only new agent Sports optics 8.75 9654.45 3.93623530789351671 +Identical solicitors must maintain sources. Factors take already unusual minutes. Just various sales sell agricultural, long states. Sports optics 3.77 1573.11 0.64137585519634677 +New hotels join increases. Agencies might not prov Sports optics 40.19 2052.76 0.83693492541071686 +Aware, single times would ring to the men. Again double months cover that. Accurate politicians send so social hotels. Other, urban feelings upset just wild eyebrows. True, magnificent p Sports optics 3.24 642.52 0.26196312685111450 +Other, international colours s Sports optics 3.14 11101.71 4.52630060541973219 +Quick artists must hope tough teachers. Social conflicts find rapidly from a shareholders; other tools Sports optics 3.81 10100.29 4.11800963472427822 +New, able officers may believe often. Losses weep fast excellent, old hours. Able, only regulations shall not let by a countries. Dreams back a little. Sophisticated, Sports optics 8.41 1446.65 0.58981659319424265 +Acute, serious forms change just premises. Above causal buildings may pay so open, traditional consequen Sports optics 4.49 7490.92 3.05413812206865251 +Ago sexual courts may attract. Important, alone observations expect. New, available ways represent years. Excell Sports optics 8.59 3179.49 1.29631628928570322 +Bombs shall not help. Angles pull sometimes. Measures train still african pictures. Teachers wear by the motives. Attractive months shall give Sports optics 0.92 NULL NULL +Other, different problems spread importantly only likely commitment Sports optics 3.10 8596.18 3.50476590888223467 +Possible opponents can inform also foreign, new heads. Losses face most qualifications. High difficulties will not walk results. Direct, ou Sports optics 0.27 149.24 0.06084694180922046 +Drugs hold years. Cells might reconsider now. Wrong players meet too rapid, integrated parents. Complete, social women used to includ Sports optics 4.94 13154.62 5.36329668763339318 +Holidays will find soon so international expectations; furious children would not talk in order reasons; there current stones shall give as firms. Central drugs ought to love european, following Sports optics 9.08 13906.80 5.66996951455686841 +European nights accompany however expensi Sports optics 1.37 3255.97 1.32749810454682075 +Earnings used to connect of course. Only big branches show into the men. Tiny trousers mediate. Highest proposed m Sports optics 8.81 3903.78 1.59161802798176516 +Wild, other services change less at a hours. Inherently southern days would win almost remarkable, separate firms; strong, professional children might damage other fea Sports optics 1.25 10597.58 4.32076074496487887 +Industrial, sexual minutes must cure crowds. Sports optics 3.33 503.37 0.20522999931993635 +Sad recordings will borrow most long teachers; then bold shares show markets. Common, dark skills watch really to a le Sports optics 8.63 838.35 0.34180537165478404 +National, little grounds must not hate broadly. Teachers define abroad normally tall researchers. Cultures handle centres. Major addresses used to look Sports optics 1.61 12110.40 4.93755564249787867 +Excellent, difficult relations attempt. Boots dismantle really social sheets. Literary sp Sports optics 1.67 2628.08 1.07149980454285779 +Obvious clubs should finance at leas Sports optics 5.51 1283.02 0.52310267542258128 +Alleged books ought to go altogether different databases; artists will listen years. Forward cold others check effectively. Quite numerous d Sports optics 5.42 3201.52 1.30529818507809887 +Teams judge conscious shareholders. Else local areas imagine ea Sports optics 2.39 6080.10 2.47892985053766615 +Tall students should encompass much true women. Rough birds ought to protect as possible families. Political, dead proceedings Sports optics 1.06 5878.74 2.39683295826545608 +Natural, political manufacturers must not pr Sports optics 2.60 1879.45 0.76627435528906048 +Physical, nationa Sports optics 52.14 5315.52 2.16720139457080890 +Rules share briefly ago specific subsidies. Maybe new subjects should scor Sports optics 1.12 NULL NULL +Exchanges see with a costs. Possible controls achieve yet high similar machines. Rights would not sum suit Sports optics 4.85 337.31 0.13752534134057995 +Legal, local prices ask central instruments. Structures cover for a parents. International tourists should Sports optics 1.84 3778.91 1.54070702809086890 +Wings can go yellow, expected eyes. Sports optics 8.93 5543.20 2.26002926719961695 +Hot grounds shall pass. Impressive methods could change very basic voices. Concrete, desirable centres pay again in a ingredients. Positio Sports optics 1.04 2610.25 1.06423029923289799 +Small aspects can allow obvious, redundant colours. Past, sound individuals give both; soft, religious months improve; customers use once for a fore Sports optics 0.82 1475.16 0.60144046287382504 +Injuries answer so good issues. Aside aware definitions m Sports optics 1.71 6407.03 2.61222314111451179 +Scenes should not learn. Magistrates produce somewhat on a businesses; extremely national values see everywhere. Northern engines shall not aim; rom Sports optics 1.88 6498.82 2.64964702739612762 +Colonies give. Even formal payments may follow comparative, frequent years. Perhaps residential messages face times. Late houses talk then conditions. Officials may includ Sports optics 76.62 15211.44 6.20188692384379802 +Great structures should not survive even here various areas. Cultural results choose likely, female hours. Gold feelings ou Sports optics 9.72 3879.70 1.58180032254913297 +Social cases need. Inc, right products can know states. Whole, economic years should run relatively new notes. Markets can stop just keen words. Now common services abuse only new, narrow feelings. Ye Sports optics 0.97 8141.82 3.31951787564424615 +Only economic shares last too white patients. Ever environmental markets might come slightly w Sports outdoor 1.07 1920.21 0.69563739953531432 +Strict results wonder indeed ago possible factors; wrong tables survive for example known differences. Featur Sports outdoor 3.18 7506.80 2.71949986242738947 +Total, happy arrangements control indeed. Particularly internatio Sports outdoor 4.20 5584.92 2.02325746945009538 +Easy, local stages may not get elected, alone pages; clean mem Sports outdoor 1.93 11116.50 4.02719137590905246 +Public questions call under way far essential taxes; Sports outdoor 1.23 9780.48 3.54318937689479327 +Preliminary, central jobs would attend unhappily personal members; as blue duties must sound remaining, slow voices. Bad years can seem short drugs. Major problems fit more middle countries. S Sports outdoor 3.62 276.60 0.10020430302491287 +Houses decide quite. Elements cannot assume simply; simple, cruel days could know. Sports outdoor 7.17 NULL NULL +Principles take hardly perhaps financial women. Men revive so in a classes. Only domestic miles perform relations. Urgent, male developers relax major po Sports outdoor 2.50 7845.25 2.84211065909688245 +Costs use again successfully coming weeks. Processes can stress less heavy, oral issues. Personally cheap officials shall go current events. Natural parties imagine powerfully without the we Sports outdoor 4.07 3610.83 1.30810088030168523 +Ago natural taxes could protect rats. More local days shall tend closely. Proteins may intervene very perfect men. Procedures make expens Sports outdoor 8.79 12330.06 4.46682960432160944 +European Sports outdoor 29.44 11343.15 4.10930021640289375 +Numbers choose special bodies. Main pictures offset like a changes; beautiful, large elections must suspend. Electronic p Sports outdoor 5.79 6902.40 2.50054295444381268 +Yet green experiments think wonderful minutes. Scottish years may remove twice parental features. Good boundaries look please. French, e Sports outdoor 8.75 3992.78 1.44647048818442374 +Good products may say pp.. Substantial, front flats become actually. Bills tr Sports outdoor 9.06 3258.39 1.18042190503740363 +Modern personnel would keep Sports outdoor 0.48 6309.82 2.28586809585197296 +Initial, real signals keep perfect, free sectors; just funny deposits can understand sufficiently. Entire relations shall not relate; poor views must reach probably. Years Sports outdoor 2.66 17724.56 6.42110333052512525 +Unacceptable events must not persuade at least but for a companies; horses would try also crude skills. Turkish, new animals go further scottish lands. European elements believe Sports outdoor 9.19 702.52 0.25450298973630437 +Eyes should jump rapidly closer explicit things. Green, radical children could ensure middle consumers. Likely minutes think very pa Sports outdoor 2.37 8733.77 3.16399615195189179 +So competent candidates would enter suddenly almost cold situations; eyebrows could read enough rational sales. Impossible Sports outdoor 0.33 2072.27 0.75072440719246635 +However subsequent steps share terribly existing communications; less great responsibilities speed at all long-term mountains. Of Sports outdoor 4.39 3486.57 1.26308502096012459 +Industries give much proposals. Possible, strong goals ought to live most total criteria. The Sports outdoor 96.84 5462.95 1.97907121189424352 +Only single galleries discover in the countries. Clean front products ought to shoot even. Ready, educational questions ought to sense shortly tests. Sciences stop. Upright variou Sports outdoor 1.53 1332.46 0.48271231239542806 +Economic elements used to hear as Sports outdoor 0.40 396.48 0.14363341309948465 +Social, joint functions should suit. Best absolute goods might not lose still western wonderful hundreds. Inches feel certain years. Diverse lives put breasts; very good police shall Sports outdoor 5.91 1973.74 0.71502979411565989 +Trees work Sports outdoor 3.30 8407.66 3.04585578586565052 +Steps cannot stay only able transaction Sports outdoor 6.89 702.30 0.25442329000143278 +Stars divorce there s Sports outdoor 2.51 7314.38 2.64979157613652275 +Original women shall know here necessarily national goods. Accounts will make as. Independent members will find a little dreams. Short jobs assist widely new moments. Ago passive represen Sports outdoor 9.83 5957.43 2.15820723416379853 +Distinctive things used to pick today symbolic pictures. Helpful lips know still. Concerned theories must accommodate very in the ph Sports outdoor 27.94 9643.98 3.49373931412219527 +Even short boards can expel anywhere secure charming details. Specia Sports outdoor 6.91 8327.04 3.01664945575043550 +Ideas form on the needs. Firstly rough operations might begin worldwide obvious activities. Twins Sports outdoor 4.30 2362.14 0.85573605331622446 +Creative teachers may close concerned, foreign parts. Alone desirable fires put pupils; areas begin behind a countries. Kindly able rates lead employers. Songs point thoroughly; large, acute others sa Sports outdoor 2.27 10905.96 3.95091872963694416 +Obviously base children must seem most for a years. Just available Sports outdoor 5.16 5010.90 1.81530637030924041 +Always small authorities make after a nations; forms will retrieve now. Financial, giant words render american, sensitive activities. Written eggs might not grant now really existing entries; grounds Sports outdoor 6.44 4934.08 1.78747667197817097 +Apparently realistic minutes see. Ful Sports outdoor 2.79 3360.22 1.21731201413728388 +Less social teeth play instead as social children. Advances mean very now slow bases. Small fit managers must think about sites; full, civil weap Sports outdoor 96.73 8555.01 3.09923649465350631 +Moreover overall miles say. Leaves may order faintly sure trees. Political, certain drinks protect to a parents. New minutes remember satisfied, exciting feet. Cri Sports outdoor 5.71 3006.51 1.08917295403987994 +Alone healthy sales might meet far other roots. French groups look up to a workers. Fully average miners would walk inadequate considerations. Small, sure goods may admire more app Sports outdoor 0.48 1427.56 0.51716433415128205 +True champions get all the same police. Especially clear issues move further great homes. Better environmental sessions burn. Bonds shall test already elderly areas. Imperial, close schools press Sports outdoor 1.71 724.38 0.26242224521036292 +Public, great addresses must prefer thick maybe dangerous problems. Public pages may shoot now injuries. Flat groups know rather special responsibilities; nuclear months can see dou Sports outdoor 9.74 6478.02 2.34680216587652229 +Quite significant levels move chiefly dirty, actual beliefs. Away significant views bury. Practical proceedings build a bit. Funds think about prime s Sports outdoor 9.44 3562.95 1.29075531982145086 +Independent, different attitudes include greatly other, bottom waters. Twin others should exert. Extraordinary, bottom tables could go only results. Good, early pupils shall say per Sports outdoor 98.21 5097.92 1.84683123816617431 +Theories must not Sports outdoor 0.92 453.25 0.16419956741157541 +Great, possible children used to Sports outdoor 4.00 8014.65 2.90347945494800407 +Truly growing visitors shall not receive open, personal times. Large societies Sports outdoor 12.35 2130.34 0.77176151448334375 +So Sports outdoor 2.12 6574.51 2.38175774504815585 +Very major companies would not remedy ever future, clear movies. Famous, equal fees know open, active rights. Original hours apply so. Social, technical rates could Sports outdoor 3.18 1551.09 0.56191573528167788 +Social thousands choose especially blue claims. Social, right professionals can go tons. General projects must ma Sports outdoor 0.64 1598.82 0.57920695503359072 +Prominent, regional tonnes ought to replace extremely. Women could make very young, equal hours. Q Sports outdoor 4.73 NULL NULL +Most whole councils arise already so social customers. More sc Sports outdoor 2.11 1583.53 0.57366782346001546 +Various pockets can get. Areas conduct photographs. Ever Sports outdoor 1.85 1513.96 0.54846459366448694 +Scientific risks would use. Quiet minutes imagine times; arms cut inner appeals. Areas happen straight in a changes. Fears kick very currently silent Sports outdoor 4.22 474.41 0.17186523282013346 +Clothes realise almost necessary females. Foreign, cultural others may give bad ya Sports outdoor 7.21 4335.56 1.57064992054479841 +Heavy years could come much through a genes. Dealers come so sincerely educational characters. Studies must handle Sports outdoor 2.12 7347.30 2.66171755464548924 +Various, personal benefits must not remember at le Sports outdoor 0.34 6983.49 2.52991955217443519 +Losses try a little cho Sports outdoor 4.86 1698.82 0.61543410724794823 +Industr Sports outdoor 8.35 1902.72 0.68930127061302319 +Nearly cultural sheets might decide to a years. Loudly new marks create lives. Local, new arrangements must not face b Sports outdoor 1.39 431.65 0.15637450253327419 +Also religious bits might hear so extensive western talks. Sometimes complete settings mean also minutes. Other, available theories admit both just old years. Considerable seconds will prepare che Sports pools 0.62 10914.03 4.26659608077049963 +Other sports take prime tables; sources think in a priests. Fine, key eyes keep always important branches. Still local effects shall get much; black, final metho Sports pools 2.25 1716.96 0.67120713492996785 +Factors would impose that is free, liable thoughts; significant wives buy useful sports; russians make nearly outstanding animals. Problems write. Finally per Sports pools 2.04 10920.36 4.26907065278388765 +Popular systems associate evenly public rights. Unlike mothers experiment around languages. Chea Sports pools 8.52 3232.70 1.26375180848016674 +Subsequent feet can accept regardless. Individual, following arms hold prime officials. Assistant acids might not get however necessary times. Sometimes new times shall not take about. Small Sports pools 1.90 9375.14 3.66500143216343934 +Bonds will set ever into the nations. Distinguished, philosophical employees may not include. General, existing tiles must continue only quiet missiles. Small ve Sports pools 12.34 9502.98 3.71497762271502301 +Western products become grea Sports pools 8.19 12699.99 4.96477722342934165 +Very old circumstances explore fairly upon a lines. Crucial, active looks mean alone bloody recordings; poor bacteria could not transfer both at a properties. States could not understand really at a Sports pools 3.35 2713.46 1.06076653640566500 +Years ought to know then. Associated, simple activities would not indicate now for a brothers. Workers get organizations. S Sports pools 20.43 4211.26 1.64629796794635660 +Supreme injuries could think conditions. Basic, eventual c Sports pools 9.13 3177.04 1.24199277557887491 +Able systems merge from a areas. Most chief efforts must find never for the time being economic directors. Activities sit there. Available polic Sports pools 3.10 4811.17 1.88081937340474643 +Carers get m Sports pools 5.77 4684.53 1.83131229603105623 +Privileges cut perhaps reasons. Ideas finish times. Women envy general programmes. Hands shall unveil never to a facilities; official proposals conform. Scot Sports pools 7.52 8558.76 3.34585591868955110 +Central, clear awards announce. Single, very proposals help dry maps. New questions Sports pools 2.90 2934.22 1.14706772403213253 +Able troubles dust into the styles. Independent feet kill wounds. Fundamental months should exploit arms. Massive years read only modern courses; twin forms shall become products. Even h Sports pools 6.81 6802.61 2.65932832922487921 +Far good grounds change clearly rocks. Growing, Sports pools 1.99 5753.89 2.24935468595785151 +Secret, familiar questions ought to influence historical values. Central, net investors can hope. So chief arrangements shoul Sports pools 6.13 4628.51 1.80941252917639637 +Fine, high letters see now suddenly prime forces. Things used to know temporary men. Late, special methods provide fr Sports pools 2.85 2565.78 1.00303434131290940 +Directors could involve. No longer local patients see waste lovers. Only direct aims canno Sports pools 60.43 1100.10 0.43005950583383284 +Similarly direct changes can alienate men; ways surrender forms. Players must develop deep. Social, serious thousands walk. Thanks will not say organisations. Natur Sports pools 3.39 3166.29 1.23779030336024597 +Simple, environmental rights ought to detail thick disabled days; also old drinks move to a conditions. Sports pools 8.46 825.24 0.32260913243733498 +Previous, significant flats give all formally co Sports pools 2.82 6467.74 2.52841838765722572 +Dangerous, other ladies may know neatly. Effortlessly growing services might encourage in the citizens. Banks use secondly other, similar responses. Indirect branches shall not buy i Sports pools 4.74 1246.28 0.48720530945422161 +Literary, sensitive pages could not know now; very public program Sports pools 3.36 2399.19 0.93790970439184930 +Christian, red laboratories prevent; shoes allow most to a positions. Now religious passengers will not know always in a elections. Southern ages abandon northern terms. Thoughts go as Sports pools 2.22 6752.13 2.63959430154149417 +Things used to reappear. Good powers lead. Rare, traditional months may pay too. Shows tend anywhere extra pp.; canadian, proper questions can investigate only small, certain countrie Sports pools 4.95 478.95 0.18723479712672870 +Like records start clear, likely un Sports pools 0.52 127.98 0.05003092042233790 +Problems might introduce therefore now public details. Early future children shall annoy ever sharp services; civil lines must fly. Finally other serv Sports pools 4.38 3165.54 1.23749710762406255 +Exclusive, different friends find for the features. Procedures comprehend totally ey Sports pools 3.90 7853.37 3.07009946489432581 +Direct, different traders woul Sports pools 4.53 3602.83 1.40844585892492317 +Southern hours see Sports pools 7.73 2352.82 0.91978238934274937 +Unable centuries may think away individuals. True, additional feet appear generally recent, pri Sports pools 3.10 741.45 0.28985330479092388 +Basic levels look early, video-taped rights. Employees might not prevail later. Causal, permanent arms could not know here public vessels Sports pools 13.28 4827.92 1.88736741151284270 +Thus aware parties would conduct either at the poems. Things plan. Instead old organizations should show rather necessary, b Sports pools 77.38 4657.72 1.82083152578161976 +Thoughtfully fine Sports pools 4.43 6849.91 2.67781920698684657 +Types can scratch like a Sports pools 9.69 3733.27 1.45943846136194267 +Only sexual functions would avoid special pati Sports pools 8.64 4120.56 1.61084083025057563 +Still male versions will get in a colonies. Wide wages would com Sports pools 1.46 5664.01 2.21421810893363108 +Then available police rememb Sports pools 0.40 1103.32 0.43131829286118030 +Pressure Sports pools 5.42 3879.88 1.51675236387107660 +Consumers remind related, slight customers. Large purposes like with a systems; types must go programmes. Main followers shall reduce al Sports pools 15.70 1464.58 0.57254481506600755 +Final holes agree really probably clear children. So good feet must imply birds. Newly british forces ought to raise nevertheless supreme, fine problems. Necessarily good units may push only Sports pools 2.20 1319.87 0.51597367508853827 +Men make only. Flat, distant depths would assert local, Sports pools 7.24 10909.61 4.26486818056525871 +Apparently other offenders should approach Sports pools 0.36 15958.64 6.23867360438145453 +Workers relieve fast quite female photographs. Other, automatic shares want away right games. Sports pools 1.82 3069.94 1.20012442445188328 +Here ready critics stay services. Excellent years ought to Sports pools 55.17 2208.60 0.86340280391291993 +Never future depths c Sports pools 23.19 4555.50 1.78087090157806155 +Real ships suspend for instance worth the arms; ago econo Sports pools 3.46 38.42 0.01501944024555573 +Famous, busy shoes will not secure. Dark, extraordinary thousands might not look then. Numbers ought to e Sports pools 6.47 7750.63 3.02993555831368042 +Massive, military measures must get standards. Services make as well fine Sports pools 0.51 10656.29 4.16583838871194852 +Critics shall not print still black parents. Multiple, accessible responses exclude against a areas. Mo Sports pools 6.14 4995.43 1.95285170187028778 +Forces eliminate away. New, large characteristics should reconsider right, used firms. Peculiar principles establish degrees. More growing arts may not say about. Actual animals move here Sports pools 2.65 1461.99 0.57153231245705415 +Senior disputes can bring tonight controversial houses. Heavy, real examples should not offer nearly free effects. Worlds will not add. Agricultural, rare defendants draw maybe possibl Sports pools 3.45 7092.42 2.77262307096263314 +Free plans ca Sports sailing 0.98 6984.42 2.34770798957927730 +Special thousands take so reforms. Finally reliable acids used to go pale, small days; great, foreign judges show vice versa fair, true arrangements Sports sailing 0.90 11949.72 4.01671908579886112 +References should make private women. Additional, northern values ar Sports sailing 0.63 14040.42 4.71947652218060722 +More critical photographs balance just now serious values. Scottish, practical views suppl Sports sailing 5.19 2863.69 0.96258642703020159 +Quite british tonnes could buy successfully surprising processes; local interests used to suggest suddenly other solicitors. Shares return just real, royal companies. Crucial, old groups study. Child Sports sailing 95.70 6541.62 2.19886741329868364 +Then other rates may make more at once above councils. Camps could give Sports sailing 0.61 8648.26 2.90698284151853421 +Scottish, british colleagues enable about a workers. Most good persons could read with a years. Indeed specific damages believe organisations. Immediate facilitie Sports sailing 1.74 7276.84 2.44600058514380124 +Easy, natural leaves contin Sports sailing 1.73 12739.66 4.28224556463149924 +New routes cannot test over a others. Armed, brown fans make so in a techniques. Electronic, subsequent professionals used to follow in a matters. Enough substantial standards Sports sailing 3.07 5349.42 1.79812727092803377 +Open times ought to add actually soviet attitudes. Women must imagine of course inner streets. Rightly big records enable yesterday st Sports sailing 6.43 2470.80 0.83052234840580583 +External, definite securities might know then particular others; always local years must buy right children. British effects used to enable powerful, Sports sailing 5.35 NULL NULL +Important, broad investors can see dearly vulnerable troops. Eastern, poor lists need genuine facilities. Figures meet equally children. Other, defensive changes go old, new companies; Sports sailing 71.43 17348.99 5.83160268628332577 +Young, black boys spread too wealthy, major numbers. Profitable drawings might think better purposes. Industr Sports sailing 3.24 12918.54 4.34237339273690257 +Joint texts take only local, asleep shareholders. Detailed courses fast programmes. Soft students know settlements; just b Sports sailing 4.70 1007.64 0.33870306748730216 +Only american aspirations will not provide then on a subjec Sports sailing 9.32 2524.02 0.84841145289915090 +Equal songs will overcome slight contracts. Large, inner odds go even good women. Feet could not find hard strong models. Bloody machines see dark heads. Huge, only men make at the advis Sports sailing 2.07 2504.57 0.84187362722467586 +Prisoners raise both. Medical children sell; activities Sports sailing 1.25 8453.80 2.84161803017362852 +Benefits may hold Sports sailing 8.02 5687.08 1.91162661371688936 +Ethnic positions must buy years. Other efforts should get; common goods show exactly aware eyes; foreign, unfair fans may carry thus daily, national actions. Sports sailing 4.63 4728.78 1.58950844693799842 +Criteria shall announce far about other waves. Farmers see possibly; just english managers clean. Head files see both. Comparisons may n Sports sailing 4.18 1308.47 0.43982255836916981 +Connections present high secondary benefits. Levels could compete. Psychological students ought to wonder advanced seats. Of course rich functions would see items; unlikely id Sports sailing 9.39 2534.25 0.85185011390942748 +Well bad areas seem Sports sailing 0.39 2413.53 0.81127189717818704 +Blue, united ministers know childr Sports sailing 4.68 530.93 0.17846415348838210 +Dear, continuous problems Sports sailing 5.90 8982.06 3.01918470322237831 +Prices acquire more out of a christians. Efficiently local prices Sports sailing 2.11 8027.95 2.69847494207721747 +Good, capable studies might like bad apparently new years. Modest, payable plants could feed there english women. New, local recommendations last public novels. Candidates must save as orange pla Sports sailing 4.28 1617.69 0.54376222186845881 +Mothers may not obtain p Sports sailing 9.99 205.80 0.06917658220087212 +British figures can tell much white methods. New, french men could think marginally nuclear relatives. Electronic, differ Sports sailing 7.39 13316.13 4.47601730584304808 +Real appearances could join miles. A Sports sailing 2.44 1182.16 0.39736534700963551 +At present financial areas used to link very purposes. Likely members can retaliate true, blac Sports sailing 1.69 7800.18 2.62191347401165555 +Special birds will not answer especially then public walls. Most human areas could require major groups. Particularly diverse children could continue to the readers Sports sailing 4.71 7976.59 2.68121104867664997 +Students would rise broad obligations. Good, statistical children would not see. Gradually elegant cases can look w Sports sailing 4.63 391.82 0.13170441417855061 +Reliable stages cannot see similarly. Feelings repeat together significant, available notes. Rich, basic roots provide instinctively before the talks. Parties arrest there other investigations. Bom Sports sailing 7.89 7983.29 2.68346315315063365 +Demands can imagine also purely fresh eyebrows. Busy skills become almost; complete pa Sports sailing 4.98 12443.47 4.18268574013161433 +Proper applications stand now very limited arms. Angrily slow boys shall aid too previous old masses. Mechanical contents think through the times. Sequences may not agree. Old, working stren Sports sailing 0.63 679.89 0.22853482250996573 +Successful, able hearts cite then contents. Urban rights will use long important, suspicious ideas; police speak for a methods. Plans seek no longer good gardens Sports sailing 4.39 8675.35 2.91608873856334289 +Scientific packages make banks. Then important parents must get front, little bact Sports sailing 4.23 6135.42 2.06232937787597103 +Also long ways should not give only now good resources. Previous, economic units s Sports sailing 4.65 389.74 0.13100525338662731 +Social years attend. Bloody wee Sports sailing 1.94 3178.08 1.06826390845941533 +Capital, foreign problems Sports sailing 3.60 1277.78 0.42950657533834004 +Original, major nations should come once more now permanent feet. Prizes revise thus with the spots. Aside ordinary studies can learn l Sports sailing 1.46 7468.82 2.51053178169833686 +Industrial, open sites would throw before a men. Also p Sports sailing 7.20 1089.19 0.36611487642064095 +Loose patients used to look at all companies. Old, low centres may illustr Sports sailing 6.35 7701.71 2.58881426094401766 +Especially moral students used to keep guilty, bizarre things. Unknown trends reduce later terms; general mothers can find as right n Sports sailing 3.35 12086.74 4.06277630296680815 +Origins would come sales. Educational eyes could invite actually stupid, forei Sports sailing 3.77 9292.44 3.12351428331716300 +Legal, secondary sales elect. Big years appeal low with a doubts. Military videos might describe; comparable, long companies would not extend now industrial tools. Even ne Sports sailing 5.45 1828.50 0.61462284039987695 +Additional organisations will adopt usually schemes. Conventional problems should not create attacks. Generally european powers win very human, busy months; fai Sports sailing 0.87 6498.29 2.18430268391693540 +Wrong, local indians train excellent, comprehensive holidays. Meals c Sports sailing 60.65 1510.40 0.50769829813506926 +National shareholders learn. Effective proceedings will develop now other, informal days; new, big waves last americans. Solicitors ought to sue flames; interested conservatives might understand just Sports sailing 0.24 5784.43 1.94434935558887624 +Ambitious exceptions appoint. V Sports sailing 7.35 9044.55 3.04018977912972767 +Proceedings mi Sports sailing 7.11 4105.60 1.38003584005782598 +Again standard families change literally. Narrow lips work certainly carefully vast stages. Drugs see also right factors. Financial, western examples ought to let desperately ago sudden Sports tennis 9.39 6556.29 1.81601129267527792 +Late global concepts shall understand very quiet, upper heads. Already english buildings make women. Others try. Please minimal agreements conflict largely forthcoming police. Sports tennis 4.33 7426.08 2.05693237186122454 +Seriously complete characteristics make forward in a projects. Industries should rise then also new departments. Physical babies encourage under to a workers. Personal, beautiful ministers cont Sports tennis 0.82 14172.38 3.92557408596710262 +Whole, new meetings may last; free plans broaden there mammals. Public, honest areas may risk on a profits. Good, normal generations ought to walk almost over a reductions. Otherwise basic s Sports tennis 4.88 8723.48 2.41629613568450044 +Economic, content activit Sports tennis 5.07 16087.57 4.45605804375706699 +Women would come fair unaware, current bars. Villages may go then on a neighbours. Early numbers should not change however cr Sports tennis 2.92 13912.86 3.85369025369685708 +Women should leave also annual, marginal techniques; intellectual, appropriate factors could think profits. Neverthe Sports tennis 8.24 23633.13 6.54608489881669218 +Of course equal nee Sports tennis 3.49 11949.65 3.30990534944566741 +Free representatives can fall much prime, useful banks. Recent, secondary practitioners can talk times; libraries take from now on young prices. Bodies appear only yellow rates. Second Sports tennis 6.85 7304.83 2.02334762054045053 +Costly offices collect officially for a debts; readers greet. Women get by a write Sports tennis 3.22 2864.47 0.79342278446035080 +Rapidly main banks shall not bring extremely decades. For example main clothes might not see less. Certainly co Sports tennis 3.15 5004.38 1.38615140465694887 +Just able pounds should join then successful modern pieces. Associated, sorry clubs pay close issues. Resources will e Sports tennis 7.67 7567.71 2.09616213128028617 +Necessary times believe probably. Cruel traders know ho Sports tennis 92.95 7731.85 2.14162688247032202 +Funny, armed savings go yet thin Sports tennis 3.97 3362.82 0.93145957473422897 +Elected, marvellous advisers may not pass all in a programmes. Directly soviet studies could not stress more than; convenient, public Sports tennis 4.67 18.70 0.00517966886349257 +Men could remove only; economic, clear children raise public, extensive poli Sports tennis 5.04 2721.49 0.75381909172761457 +Able, common villages read. Only social grounds remember e Sports tennis 2.08 2677.23 0.74155961879188295 +Successful parties see once on a ideas. Scottish, natural men would not examine regulatory, multiple payments. Steadily loc Sports tennis 2.55 8031.03 2.22449604453340795 +Current, Sports tennis 0.47 18310.05 5.07165753336856247 +Years may speak to a Sports tennis 2.02 3056.11 0.84650469574375807 +Separate, comfortable consumers get. Tests work even high, different faces. Hars Sports tennis 8.09 11878.41 3.29017274998923903 +Much critical possibilities might ensure; hence northern ways may persuade much japanese, running notes. Small, ed Sports tennis 8.53 8171.42 2.26338233927916847 +As specific ears worry also labour components. Duly proper articles would attend more easy shapes; years wait head convention Sports tennis 0.85 11273.32 3.12257029904748936 +Early, experimental factors mean usually suitable detectives; just black assets must not store only. So british employers must see elaborate, complete pages. Mental years should t Sports tennis 88.56 15092.59 4.18046088194969605 +Social, substantial orders would not offset however to a colleagues. Small students give for sure husbands. Subjects shall not make generations; acceptable lights g Sports tennis 56.30 5682.58 1.57400442194147617 +I Sports tennis 1.04 4973.48 1.37759248658839698 +Automatic amounts may find more in a regulations. Boys can give available, current seasons; here complicated camps may spot even generous open individuals. Channels remain currently protest Sports tennis 8.43 3330.22 0.92242977767808685 +Points used to find cool typical managers. However military horses understand indeed inc periods. Coloured developments could make very roots. Sports tennis 8.52 11481.61 3.18026405453288334 +Sides express even new women. Also joint markets should switch authorities. Trees would play always about a skills. Teams deprive future pubs; ways recover national, old days. Rea Sports tennis 90.25 3634.02 1.00657862263685918 +Secret children will start in short familie Sports tennis 38.21 13612.04 3.77036683190456646 +Other, general countries keep successfully teachers. Major, traditional relationships could not become in a subjects. Constant observers wil Sports tennis 99.16 7979.51 2.21022564133302628 +Upper, industrial years shall opera Sports tennis 1.58 369.36 0.10230815462136981 +Afraid, spanish matt Sports tennis 3.06 141.37 0.03915774263272431 +Light, social animals resist instead then female societies. Also informal minutes shall not implement. Servants win. Hands will a Sports tennis 8.30 3341.21 0.92547387183903783 +Modest, educational principles would Sports tennis 6.42 18707.39 5.18171580215038800 +Far little eyes can happen pp.. Related margins will suffer low below active children; times feel just similar, nervous birds. Vegetabl Sports tennis 9.01 813.78 0.22540700148304722 +Then various shoes date good, bad shops. Here open rats match badly well dual games. No doubt small kids answer much points. Completely free services shall understand. Following patients Sports tennis 5.46 1154.69 0.31983485775327459 +Widely free parties would find in a problems. Men like parties; straight a Sports tennis 8.95 10297.10 2.85216942536199653 +Tired rights free. Paintings sell Sports tennis 8.06 5429.22 1.50382683353214583 +Meetings improve early women. Even likely variables might want approxi Sports tennis 2.56 7342.79 2.03386207134570068 +Growing jews see only grey tactics. Also indian parts ought to provide pretty other, canadian ways. Countries shall correspond really to a doubts. Star sounds ought to mean further at a steps. Sports tennis 8.04 4423.03 1.22512464028307694 +Else single arrangements will not keep approximately from a teachers. Large levels tolerate daily financial, critical others. Properties make a Sports tennis 0.30 5475.18 1.51655718545546767 +Equivalent, important points would not reject foreign, high mountains. Always alive cups mark near the games. Sons will not stay extremely. Unfortunatel Sports tennis 0.19 5314.97 1.47218099568968454 +Confidential companies can write highly; potentially new children mix sympathetically military, economic gains. Various, traditional designers make in a measurements. Individuals tell only se Sports tennis 7.12 1837.86 0.50906450360740392 +Examples show waves. Currently representative farmers should put like a customers. Both full rights practise with a police. Legal re Sports tennis 4.24 735.27 0.20366070188557120 +Part Sports tennis 6.53 4928.46 1.36512250304644856 +Great, big arts will not let brilliant pp.. Real, new or Sports tennis 0.88 13772.83 3.81490367450140978 +Inc presents cannot break often subjects. Of course capital services would pay. Systems cannot Sports tennis 9.67 3395.45 0.94049768141956387 +Parts may refuse primarily old holidays. Scottish, good tests handle however for the households; low measurements will remember into a calls; inc, genuine events used to think again r Sports tennis 6.88 733.87 0.20327291918990865 +Literary pai Sports tennis 2.68 3317.04 0.91877908058606374 +Themes would not reflect on the jeans. Traditional relations would not force mildly smal Sports tennis 9.89 1274.76 0.35309276365913303 diff --git a/sql/core/src/test/resources/tpcds-query-results/v1_4/q99.sql.out b/sql/core/src/test/resources/tpcds-query-results/v1_4/q99.sql.out new file mode 100644 index 0000000000000..2a7da81704380 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v1_4/q99.sql.out @@ -0,0 +1,95 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct120 days :bigint> +-- !query output +Just good amou EXPRESS Mid Atlantic 1260 1337 1352 0 0 +Just good amou EXPRESS NY Metro 1297 1268 1203 0 0 +Just good amou EXPRESS North Midwest 1291 1266 1327 0 0 +Just good amou LIBRARY Mid Atlantic 932 1025 970 0 0 +Just good amou LIBRARY NY Metro 918 973 922 0 0 +Just good amou LIBRARY North Midwest 899 945 970 0 0 +Just good amou NEXT DAY Mid Atlantic 1273 1356 1306 0 0 +Just good amou NEXT DAY NY Metro 1301 1304 1226 0 0 +Just good amou NEXT DAY North Midwest 1158 1250 1247 0 0 +Just good amou OVERNIGHT Mid Atlantic 1021 961 1055 0 0 +Just good amou OVERNIGHT NY Metro 852 992 977 0 0 +Just good amou OVERNIGHT North Midwest 858 993 982 0 0 +Just good amou REGULAR Mid Atlantic 948 982 993 0 0 +Just good amou REGULAR NY Metro 910 941 951 0 0 +Just good amou REGULAR North Midwest 880 948 993 0 0 +Just good amou TWO DAY Mid Atlantic 958 1030 981 0 0 +Just good amou TWO DAY NY Metro 891 907 907 0 0 +Just good amou TWO DAY North Midwest 924 886 955 0 0 +Matches produce EXPRESS Mid Atlantic 1216 1329 1314 0 0 +Matches produce EXPRESS NY Metro 1164 1301 1225 0 0 +Matches produce EXPRESS North Midwest 1246 1265 1264 0 0 +Matches produce LIBRARY Mid Atlantic 890 962 963 0 0 +Matches produce LIBRARY NY Metro 891 970 963 0 0 +Matches produce LIBRARY North Midwest 956 902 964 0 0 +Matches produce NEXT DAY Mid Atlantic 1339 1308 1245 0 0 +Matches produce NEXT DAY NY Metro 1218 1280 1177 0 0 +Matches produce NEXT DAY North Midwest 1265 1249 1309 0 0 +Matches produce OVERNIGHT Mid Atlantic 928 916 959 0 0 +Matches produce OVERNIGHT NY Metro 928 937 959 0 0 +Matches produce OVERNIGHT North Midwest 924 985 923 0 0 +Matches produce REGULAR Mid Atlantic 937 919 956 0 0 +Matches produce REGULAR NY Metro 920 970 942 0 0 +Matches produce REGULAR North Midwest 920 978 1033 0 0 +Matches produce TWO DAY Mid Atlantic 947 961 1010 0 0 +Matches produce TWO DAY NY Metro 870 950 1004 0 0 +Matches produce TWO DAY North Midwest 896 989 883 0 0 +Operations EXPRESS Mid Atlantic 1282 1274 1361 0 0 +Operations EXPRESS NY Metro 1183 1267 1206 0 0 +Operations EXPRESS North Midwest 1182 1297 1234 0 0 +Operations LIBRARY Mid Atlantic 955 1001 1015 0 0 +Operations LIBRARY NY Metro 917 948 930 0 0 +Operations LIBRARY North Midwest 890 926 977 0 0 +Operations NEXT DAY Mid Atlantic 1197 1322 1291 0 0 +Operations NEXT DAY NY Metro 1221 1238 1294 0 0 +Operations NEXT DAY North Midwest 1277 1295 1273 0 0 +Operations OVERNIGHT Mid Atlantic 904 1021 953 0 0 +Operations OVERNIGHT NY Metro 923 915 975 0 0 +Operations OVERNIGHT North Midwest 932 1010 987 0 0 +Operations REGULAR Mid Atlantic 953 1024 974 0 0 +Operations REGULAR NY Metro 902 892 901 0 0 +Operations REGULAR North Midwest 938 942 990 0 0 +Operations TWO DAY Mid Atlantic 964 990 1011 0 0 +Operations TWO DAY NY Metro 917 946 886 0 0 +Operations TWO DAY North Midwest 926 973 980 0 0 +Selective, EXPRESS Mid Atlantic 1307 1294 1263 0 0 +Selective, EXPRESS NY Metro 1186 1296 1230 0 0 +Selective, EXPRESS North Midwest 1203 1289 1271 0 0 +Selective, LIBRARY Mid Atlantic 1004 959 972 0 0 +Selective, LIBRARY NY Metro 932 940 931 0 0 +Selective, LIBRARY North Midwest 925 912 940 0 0 +Selective, NEXT DAY Mid Atlantic 1267 1309 1327 0 0 +Selective, NEXT DAY NY Metro 1244 1244 1261 0 0 +Selective, NEXT DAY North Midwest 1234 1268 1270 0 0 +Selective, OVERNIGHT Mid Atlantic 978 945 1068 0 0 +Selective, OVERNIGHT NY Metro 938 963 947 0 0 +Selective, OVERNIGHT North Midwest 882 936 932 0 0 +Selective, REGULAR Mid Atlantic 989 948 970 0 0 +Selective, REGULAR NY Metro 917 972 980 0 0 +Selective, REGULAR North Midwest 876 937 1001 0 0 +Selective, TWO DAY Mid Atlantic 951 974 972 0 0 +Selective, TWO DAY NY Metro 928 1007 934 0 0 +Selective, TWO DAY North Midwest 968 942 996 0 0 +Significantly EXPRESS Mid Atlantic 1260 1340 1298 0 0 +Significantly EXPRESS NY Metro 1231 1326 1236 0 0 +Significantly EXPRESS North Midwest 1200 1222 1233 0 0 +Significantly LIBRARY Mid Atlantic 949 1048 965 0 0 +Significantly LIBRARY NY Metro 908 963 915 0 0 +Significantly LIBRARY North Midwest 970 984 920 0 0 +Significantly NEXT DAY Mid Atlantic 1312 1347 1268 0 0 +Significantly NEXT DAY NY Metro 1198 1251 1190 0 0 +Significantly NEXT DAY North Midwest 1231 1232 1307 0 0 +Significantly OVERNIGHT Mid Atlantic 990 973 990 0 0 +Significantly OVERNIGHT NY Metro 891 925 954 0 0 +Significantly OVERNIGHT North Midwest 876 971 958 0 0 +Significantly REGULAR Mid Atlantic 942 1006 913 0 0 +Significantly REGULAR NY Metro 955 956 957 0 0 +Significantly REGULAR North Midwest 910 937 1001 0 0 +Significantly TWO DAY Mid Atlantic 957 1027 1018 0 0 +Significantly TWO DAY NY Metro 971 972 958 0 0 +Significantly TWO DAY North Midwest 885 977 919 0 0 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q10a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q10a.sql.out new file mode 100644 index 0000000000000..683a462c2ecc6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q10a.sql.out @@ -0,0 +1,9 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +F S Advanced Degree 1 4500 1 High Risk 1 2 1 0 1 0 1 +M D Unknown 1 5000 1 Unknown 1 5 1 4 1 6 1 +M M 2 yr Degree 1 2500 1 Unknown 1 2 1 0 1 4 1 +M W Primary 1 2000 1 Low Risk 1 2 1 3 1 4 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q11.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q11.sql.out new file mode 100644 index 0000000000000..f8a1d6edcd267 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q11.sql.out @@ -0,0 +1,99 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAAECBAAA Frank Wenzel Frank.Wenzel@zhXN6.com +AAAAAAAAABGKAAAA Jonna King Jonna.King@aFu3Mu88XNgLh7lzUbBd.com +AAAAAAAAAFAGAAAA Robert Chang Robert.Chang@q5SPPnTKPgA2siE.org +AAAAAAAAAFBNAAAA Robert Baines Robert.Baines@FI8euotCCfA0dfsoy.com +AAAAAAAAAGLPAAAA Charlene Marcus Charlene.Marcus@XYRXjq9m6.com +AAAAAAAABAAGAAAA Luis James Luis.James@oLxkv69Mc9.edu +AAAAAAAABBEAAAAA Jason Gallegos Jason.Gallegos@sg0JhLIArOU5lOS.org +AAAAAAAABGMHBAAA Michael Gillespie Michael.Gillespie@J63SDK8lTkTx.edu +AAAAAAAABIABAAAA Letha Stone Letha.Stone@BkqMc.com +AAAAAAAABILCAAAA Theresa Mullins Theresa.Mullins@96UTbTai7sO.org +AAAAAAAABJEDBAAA Arthur Bryan Arthur.Bryan@ZvCpRQMEbZYcg.org +AAAAAAAABKDKAAAA Gerald Ruiz Gerald.Ruiz@kHcL2q.com +AAAAAAAACEMIAAAA James Hernandez James.Hernandez@gj0dkjapodlS.com +AAAAAAAACGLDAAAA Angelo Sloan Angelo.Sloan@dabad6klflJ.edu +AAAAAAAACKKIAAAA NULL NULL Lorraine.Miller@31f.edu +AAAAAAAACOEHBAAA Christine Gonzalez Christine.Gonzalez@oHMdrqfEDX.org +AAAAAAAACPDFBAAA Cheryl Barry Cheryl.Barry@b4id7Q6XJNsB.edu +AAAAAAAADFJBBAAA Patrick Jones Patrick.Jones@L0aUVXsdRxldn.com +AAAAAAAADHNHBAAA Patrick Cooper Patrick.Cooper@2kXFgCYx14V.edu +AAAAAAAADKMBAAAA Donald Nelson Donald.Nelson@b6TdhXbAelMn8oF.edu +AAAAAAAAEBFHAAAA Esther Merrill Esther.Merrill@hGu.edu +AAAAAAAAEBJNAAAA Alfred Glenn Alfred.Glenn@xCi0.edu +AAAAAAAAEFCEBAAA Cornelius Martino Cornelius.Martino@In2KFInUjUY.com +AAAAAAAAEIAHAAAA Henry Desantis Henry.Desantis@znTqdvjJGag4.edu +AAAAAAAAEIPIAAAA Luke Rios Luke.Rios@NgqF4xn2Qxgm00FR0.com +AAAAAAAAFAIEAAAA Betty Gipson Betty.Gipson@13Lp7iesLn.com +AAAAAAAAFDIMAAAA Stephanie Cowan Stephanie.Cowan@R80Njmu1D1n0d.com +AAAAAAAAFGMHBAAA Donald Coleman Donald.Coleman@S4KL45.org +AAAAAAAAFGNEAAAA Andrew Silva Andrew.Silva@hx4.edu +AAAAAAAAFHNDAAAA Virgil Mcdonald Virgil.Mcdonald@dUD.org +AAAAAAAAFMOKAAAA Harry Brown Harry.Brown@Clj2rtJAo.com +AAAAAAAAFMPPAAAA Manuel Bryant Manuel.Bryant@1LtMa1H0t8B5.edu +AAAAAAAAFOEDAAAA Lori Erwin Lori.Erwin@SkmpHUaEnhHBkQ.com +AAAAAAAAGCGIAAAA Mae Williams Mae.Williams@mfBvsN8VAQOX21Yh.org +AAAAAAAAGEKLAAAA Jerilyn Walker Jerilyn.Walker@hOIXjGj8unTzQ5J3Um.edu +AAAAAAAAGGMHAAAA Julia Fisher Julia.Fisher@eyrOB7M7abp.org +AAAAAAAAGHFDAAAA Laura Roy Laura.Roy@xb1d3mQ2.org +AAAAAAAAGLDMAAAA Alex Norris Alex.Norris@GABnCVFfjXxUV2Q.edu +AAAAAAAAGMFHAAAA Bruce Howe Bruce.Howe@yNj94o0DBJ.com +AAAAAAAAGMGEBAAA Tamika Potts Tamika.Potts@yzUu.edu +AAAAAAAAHBEABAAA Bonnie Cochran Bonnie.Cochran@D3oggm81Joopv.com +AAAAAAAAHEIFBAAA NULL Jones Ann.Jones@E1eNB.edu +AAAAAAAAHEPFBAAA Kathryn Kinney Kathryn.Kinney@Stq02g.com +AAAAAAAAHGKLAAAA Arthur Christensen Arthur.Christensen@VFNBhqKt1TAdrr.edu +AAAAAAAAHIEIAAAA William Roberts William.Roberts@ObeXEfeXMMgm.org +AAAAAAAAHLEAAAAA Geneva Sims Geneva.Sims@1E0ayoK5qFo.edu +AAAAAAAAHLJCAAAA Marlene Grover Marlene.Grover@F9DZzXQsJNYJ.org +AAAAAAAAHPMLAAAA Elizabeth Kennedy Elizabeth.Kennedy@A0YBfGbsbmc3om.edu +AAAAAAAAIANDAAAA Elva Wade Elva.Wade@4xZ1agk4PU9.edu +AAAAAAAAIBBFBAAA James Compton James.Compton@KzmxhGNiTqrp.com +AAAAAAAAIBJDBAAA Dean Velez Dean.Velez@ycRqZT5hVfX8ZZk.org +AAAAAAAAILLJAAAA Billy Ortiz Billy.Ortiz@AEcBAd1rTF.org +AAAAAAAAIODCBAAA Jennifer Crane Jennifer.Crane@Sbzbbg2f7tIl32aDBj.org +AAAAAAAAIPGJAAAA Michael NULL Michael.Connelly@K.edu +AAAAAAAAIPKJAAAA Charles Jones Charles.Jones@aZTRs91tA.org +AAAAAAAAJADIAAAA Margaret Roberts Margaret.Roberts@jp.edu +AAAAAAAAJBELAAAA Sean Busby Sean.Busby@HlbL26U77.edu +AAAAAAAAJCNBBAAA Johnnie Cox Johnnie.Cox@nNTlnRXjr5.edu +AAAAAAAAJDEFAAAA Loretta Serrano Loretta.Serrano@GYZpg38p40VgqS7L9.edu +AAAAAAAAJDKKAAAA Sharon Reynolds Sharon.Reynolds@tk5.org +AAAAAAAAJEDJAAAA David Taylor David.Taylor@kxn8ngym6u9XoC.org +AAAAAAAAJGDLAAAA Fredrick Davis Fredrick.Davis@fBIx4ZgRJ2.org +AAAAAAAAJHGFAAAA Pamela Gannon Pamela.Gannon@dx1Vy6KLG.org +AAAAAAAAJIAHAAAA Shawna Delgado Shawna.Delgado@Mu5QaTkI2N4tdINV.org +AAAAAAAAJILDBAAA Erica Reynolds Erica.Reynolds@NjAGMPr5SynCgvs.org +AAAAAAAAJMIDAAAA Sally Thurman Sally.Thurman@xTyyZ3qRIlqa8oBLYTNm.org +AAAAAAAAKAKPAAAA Carolann Royer Carolann.Royer@yVh8tzAJmV.com +AAAAAAAAKLHDAAAA Brittany Knox Brittany.Knox@ldm93PY1oSCUpHZQfy.org +AAAAAAAAKMHPAAAA Robert Jones Robert.Jones@rYXHiKMN4A.org +AAAAAAAAKNMEBAAA Amber Gonzalez Amber.Gonzalez@Va5m6mBBm.edu +AAAAAAAALEAHBAAA Eddie Pena Eddie.Pena@2L00HyEmYqdOy.org +AAAAAAAALMAJAAAA Ileen Linn Ileen.Linn@5FUTo7S.org +AAAAAAAALMGGBAAA Dedra Rainey Dedra.Rainey@ik6CKcRSdO6GBi.edu +AAAAAAAALNLABAAA Janie Garcia Janie.Garcia@IpXCI4cANG0F1M.org +AAAAAAAALPHGBAAA Dorothy Heller Dorothy.Heller@dXV5.edu +AAAAAAAAMFMKAAAA John Sanders John.Sanders@g.com +AAAAAAAAMHOLAAAA Terri Cook Terri.Cook@Vz02fJPUlPO.edu +AAAAAAAAMJFAAAAA Marcus Espinal Marcus.Espinal@zoIoG4RpC.com +AAAAAAAAMLOEAAAA Miguel Jackson Miguel.Jackson@i8Me4xM79.org +AAAAAAAANBECBAAA Michael Lombardi Michael.Lombardi@J.com +AAAAAAAANKBBAAAA Diann Saunders Diann.Saunders@g6OYMRl4DEBFz.org +AAAAAAAAOCDCAAAA Armando Jackson Armando.Jackson@IoY0Kf.edu +AAAAAAAAOEDIAAAA Alexander Rich Alexander.Rich@YT4HorlXiEXVj.org +AAAAAAAAOFFIAAAA Frank Milton Frank.Milton@satakl9QHE.edu +AAAAAAAAOJBPAAAA Jonathan Mcbride Jonathan.Mcbride@SjQzPb47cUO.com +AAAAAAAAOMOKAAAA Laurette Gary Laurette.Gary@bSgl2F0kEo2tf.org +AAAAAAAAOOKKAAAA Deborah Early Deborah.Early@Zpi3TmsGBi.edu +AAAAAAAAOPMDAAAA Peggy Smith Peggy.Smith@CfqGXuI6hH.org +AAAAAAAAOPPKAAAA Tina Johnson Tina.Johnson@LlITreSC2jD7.com +AAAAAAAAPAEEBAAA Audria Mattson Audria.Mattson@V6zU0l0A.com +AAAAAAAAPBIGBAAA Susie Zavala Susie.Zavala@C0UOUuL65F7kV.com +AAAAAAAAPEFLAAAA David Martinez David.Martinez@ghefIHRjR1N.com +AAAAAAAAPFKDAAAA Linda Simmons Linda.Simmons@P3L.com +AAAAAAAAPNMGAAAA Christine Olds Christine.Olds@acdIL3Bsp4QnMIc.org diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q12.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q12.sql.out new file mode 100644 index 0000000000000..e60df322b6489 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q12.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAAELBAAAA Precisely elderly bodies Books arts 1.40 11.21 0.01417562243122168 +AAAAAAAAALNCAAAA Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 515.52 0.65190159462474560 +AAAAAAAABKLDAAAA Forward psychological plants establish closely yet eastern changes. Likewise necessary techniques might drop. Pleasant operations like lonely things; dogs let regions. Forces might not result clearl Books arts 2.43 11462.46 14.49487110552909973 +AAAAAAAADLLDAAAA Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 3400.60 4.30023386615632740 +AAAAAAAAFCIBAAAA United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 784.30 0.99178774958137022 +AAAAAAAAFFIBAAAA Naturally new years put serious, negative vehicles. Fin Books arts 3.34 3319.96 4.19826043236027781 +AAAAAAAAGHBAAAAA Hard different differences would not paint even. Together suitable schemes marry directly only open women. Social ca Books arts 2.65 229.68 0.29044219090124839 +AAAAAAAAGMFAAAAA Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 5752.44 7.27425669029944833 +AAAAAAAAHHEBAAAA Powerful walls will find; there scottish decades must not Books arts 4.16 434.76 0.54977641464745189 +AAAAAAAAIBOCAAAA Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 969.48 1.22595739827125692 +AAAAAAAAICMBAAAA Aside legitimate decisions may not stand probably sexual g Books arts 3.88 349.20 0.44158138742039332 +AAAAAAAAIFPBAAAA Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 3343.99 4.22864760515441312 +AAAAAAAAIHNAAAAA Unlikely states take later in general extra inf Books arts 0.32 20046.98 25.35043883731064290 +AAAAAAAAJLHBAAAA Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 11869.78 15.00994822673206253 +AAAAAAAAKHJDAAAA Relevant lips take so sure, manufacturing Books arts 8.80 5995.28 7.58134037907713537 +AAAAAAAAKHLBAAAA Extra, primitive weeks look obviou Books arts 1.18 425.89 0.53855984275049058 +AAAAAAAALCFBAAAA More than key reasons should remain. Words used to offer slowly british Books arts 0.28 7814.52 9.88186306879843074 +AAAAAAAALGEEAAAA Children may turn also above, historical aspects. Surveys migh Books arts 7.22 544.72 0.68882649872748182 +AAAAAAAALOKCAAAA Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 152.67 0.19305907908783347 +AAAAAAAANMECAAAA Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 503.10 0.63619586486597904 +AAAAAAAAODBEAAAA Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 922.40 1.16642231316314662 +AAAAAAAAOODBAAAA Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 230.48 0.29145383210954253 +AAAAAAAAAJJBAAAA New, reluctant associations see more different, physical symptoms; useful pounds ought to give. Subjects Books business 9.02 306.85 0.37352072221391094 +AAAAAAAABBLDAAAA Natural plans might not like n Books business 4.29 2484.54 3.02436752540117416 +AAAAAAAABINDAAAA Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 90.28 0.10989555418436330 +AAAAAAAACDDCAAAA Simple, great shops glance from a years. Lessons deepen here previous clients. Increased, silent flights open more great rocks. Brill Books business 8.92 393.75 0.47930188812686144 +AAAAAAAACGIDAAAA Groups must not put new, civil moves. Correct men laugh slightly total novels. Relatively public girls set even scott Books business 3.36 344.10 0.41886420242400767 +AAAAAAAACNEDAAAA Just young degrees stop posts. More than tight artists buy to a arts. European, essential techniques ought to sell to a offences. Sentences be Books business 2.58 184.08 0.22407591508925118 +AAAAAAAADEDAAAAA Junior, severe restrictions ought to want principles. Sure, Books business 9.77 1549.80 1.88653223166732663 +AAAAAAAAEEFDAAAA Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 6463.45 7.86779374936777799 +AAAAAAAAEGKCAAAA Russian windows should see in a weapons. New, considerable branches walk. English regions apply neither alone police; very new years w Books business 2.79 1635.60 1.99097439548011320 +AAAAAAAAEKDAAAAA Long groups used to create more tiny feet; tools used to dare still Books business 57.04 10558.62 12.85274032257534413 +AAAAAAAAEPLBAAAA Drugs must compensate dark, modest houses. Small pubs claim naturally accessible relationships. Distinguished Books business 1.66 31.78 0.03868498794837246 +AAAAAAAAFCGDAAAA Small, capable centres Books business 2.98 3219.72 3.91928349267255446 +AAAAAAAAFDLAAAAA Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 3862.88 4.70218584789203943 +AAAAAAAAFEGEAAAA Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 10904.74 13.27406341976510738 +AAAAAAAAFHFCAAAA Sound, original activities consider quite to a attitudes. In order weak improvements marry available, hard studie Books business 71.27 385.84 0.46967324575204627 +AAAAAAAAHDLBAAAA Classic issues will draw as european, engl Books business 75.64 92.64 0.11276832232653319 +AAAAAAAAHJAAAAAA Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 5353.50 6.51667976657054660 +AAAAAAAAIMJAAAAA Departments could seek now for a commu Books business 5.93 6535.44 7.95542535045032467 +AAAAAAAAJFBEAAAA Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 303.68 0.36966196161616580 +AAAAAAAAJJGBAAAA Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 2096.55 2.55207713918062566 +AAAAAAAAKKDAAAAA Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 6646.96 8.09117581791421695 +AAAAAAAAKNJCAAAA White fees might combine reports. Tr Books business 2.09 500.56 0.60931899205277908 +AAAAAAAALDHBAAAA Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 974.26 1.18594198736882801 +AAAAAAAALOMDAAAA Only new systems might join late speeches. Materials could stay on a benefits. Corporate regulations must crawl definitely practical deaths. Windows might soothe despite a organisations. Old Books business 0.67 9075.35 11.04719337247520503 +AAAAAAAAMBECAAAA Professional managers form later initial grounds. Conscious, big risks restore. American, full rises say for a systems. Already Books business 5.27 890.13 1.08353267219901759 +AAAAAAAAMKGDAAAA Memories can earn particularly over quick contexts; alone differences make separate years; irish men mea Books business 4.23 2059.92 2.50748836924516678 +AAAAAAAANJLBAAAA Only, gothic Books business 1.68 4777.17 5.81512787530920297 +AAAAAAAAOAPAAAAA Silver, critical operations could help howev Books business 5.56 428.54 0.52165087273113702 +AAAAAAAAALNBAAAA Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 2456.28 1.26602601850774711 +AAAAAAAACFMBAAAA At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 10252.90 5.28459221471415324 +AAAAAAAACOFCAAAA Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 1761.68 0.90801240749618444 +AAAAAAAADAHAAAAA Ga Books computers 5.53 7541.48 3.88706087988983530 +AAAAAAAADDBAAAAA S Books computers 65.78 4566.02 2.35343695385979752 +AAAAAAAAECHAAAAA Boxes batt Books computers 0.83 7844.04 4.04300760915510798 +AAAAAAAAEJECAAAA Artists make times. Rather ready functions must pre Books computers 5.71 3694.01 1.90398194531071494 +AAAAAAAAFDPCAAAA Limited, capable cities shall try during the bodies. Specially economic services ought to prevent old area Books computers 2.93 96.18 0.04957349425150028 +AAAAAAAAFHNAAAAA Legs throw then. Old-fashioned develo Books computers 2.66 718.55 0.37035801928067716 +AAAAAAAAFOCEAAAA Important, educational variables used to appear months. A Books computers 2.47 9922.02 5.11404867366677942 +AAAAAAAAGHEAAAAA Men should not turn shadows. Different, single concessions guarantee only therefore alone products. Books computers 8.38 4194.24 2.16181256528813215 +AAAAAAAAGIFEAAAA Educational, white teachers should not fix. Considerable, other services might not cover today on a forms. Successful genes fall otherwise so Books computers 1.65 14569.68 7.50956485471198434 +AAAAAAAAHGCEAAAA Present Books computers 2.84 12393.53 6.38792460190056468 +AAAAAAAAHHFDAAAA Multiple, dark feet mean more complex girls; schools may not answer frequently blue assets. Spiritual, dry patients may reply personnel Books computers 2.04 371.40 0.19142852739662305 +AAAAAAAAIBDEAAAA Private teachers ap Books computers 5.27 4911.39 2.53144899076602182 +AAAAAAAAIDCDAAAA Daily numbers sense interesting players. General advantages would speak here. Shelves shall know with the reductions. Again wrong mothers provide ways; as hot pr Books computers 7.56 689.26 0.35526124607807325 +AAAAAAAAIECAAAAA Inc, corporate ships slow evident degrees. Chosen, acute prices throw always. Budgets spend points. Commonly large events may mean. Bottles c Books computers 68.38 4.17 0.00214931868401701 +AAAAAAAAIOKCAAAA However old hours ma Books computers 8.84 451.53 0.23272946412330966 +AAAAAAAAJDOCAAAA Indeed other actions should provide after a ideas; exhibitio Books computers 6.95 8062.32 4.15551439149257400 +AAAAAAAAKDGEAAAA Perfect days find at all. Crimes might develop hopes. Much socialist grants drive current, useful walls. Emissions open naturally. Combinations shall not know. Tragic things shall not receive just Books computers 6.71 526.49 0.27136565802113105 +AAAAAAAAKOKBAAAA Human windows take right, variable steps. Years should buy often. Indeed thin figures may beat even up to a cars. Details may tell enough. Impossible, sufficient differences ought to return Books computers 4.47 1542.60 0.79509328584283986 +AAAAAAAAKPNDAAAA Left diff Books computers 0.74 5248.81 2.70536340572070289 +AAAAAAAAMDKBAAAA Friendly, hot computers tax elsewhere units. New, real officials should l Books computers 3.19 12378.72 6.38029117031536278 +AAAAAAAAMENDAAAA Kinds relieve really major practices. Then capable reserves could not approve foundations. Pos Books computers 7.23 1786.41 0.92075884659828053 +AAAAAAAAMJJCAAAA Visible, average words shall not want quite only public boundaries. Cars must telephone proposals. German things ask abilities. Windows cut again favorite offi Books computers 6.75 25255.89 13.01749550563031296 +AAAAAAAANANCAAAA Only increased errors must submit as rich, main Books computers 6.94 2429.79 1.25237243291071818 +AAAAAAAANFHDAAAA Meals ought to test. Round days might need most urban years. Political, english pages must see on a eyes. Only subsequent women may come better methods; difficult, social childr Books computers 7.23 6457.72 3.32846480866914548 +AAAAAAAANHFDAAAA Systems cannot see fairly practitioners. Little ca Books computers 1.73 6197.59 3.19438752586978211 +AAAAAAAANKLDAAAA Past beautiful others might not like more than legislative, small products. Close, wh Books computers 3.02 10232.30 5.27397447733028024 +AAAAAAAAOGDDAAAA Main problems wait properly. Everyday, foreign offenders can worry activities. Social, important shoes will afford okay physical parts. Very Books computers 1.40 2034.30 1.04852733786470188 +AAAAAAAAOGMDAAAA Schools offer quickly others. Further main buildings satisfy sadly great, productive figures. Years contribute acti Books computers 4.11 3975.90 2.04927485750197523 +AAAAAAAAOMDAAAAA Tiny, rare leaders mention old, precious areas; students will improve much multiple stars. Even confident solutions will include clearly single women. Please little rights will not mention harder com Books computers 1.45 11902.91 6.13504720795513872 +AAAAAAAAONDCAAAA Guidelines should investigate so. Usual personnel look now old, modern aspects. Discussions could appear once br Books computers 2.44 640.06 0.32990237815154161 +AAAAAAAAONHDAAAA Flat pleasant groups would go private, redundant eyes. Main devic Books computers 2.83 8359.41 4.30864175068552700 +AAAAAAAAOPNBAAAA Fine users go for a networks. Sympathetic, effective industries could not alter particularly other concepts; wooden women used to feel however Books computers 5.30 247.79 0.12771694885193653 +AAAAAAAAPAKAAAAA Real, domestic facilities turn often guilty symptoms. Winds get naturally intense islands. Products shall not travel a little clear shares; improved children may not apply wrong c Books computers 5.28 297.60 0.15339022550682558 +AAAAAAAAABIBAAAA Then irish champions must look now forward good women. Future, big models sign. Then different o Books cooking 85.81 6496.48 10.66582432143788856 +AAAAAAAAAGHBAAAA Valuable studies should persist so concerned parties. Always polite songs include then from the holes. There conventional areas might not explain theore Books cooking 1.58 2088.03 3.42809662430915734 +AAAAAAAAAIJCAAAA Meanings occur in a things. Also essential features may not satisfy by the potatoes; happy words change childre Books cooking 3.46 1496.40 2.45676728237440221 +AAAAAAAAAJDBAAAA Then dominant goods should combine probably american items. Important artists guess only sill Books cooking 6.67 5638.06 9.25648312220250073 +AAAAAAAADDNAAAAA Individual, remarkable services take by the interest Books cooking 6.05 0.00 0.00000000000000000 +AAAAAAAAEGFEAAAA Ultimately senior elections marry at l Books cooking 5.06 2060.48 3.38286544372280691 +AAAAAAAAEINDAAAA Hence young effects shall not solve however months. In order small activities must not return almost national foods. International decades take contributions. Sessions must see Books cooking 1.43 242.28 0.39777170353760369 +AAAAAAAAENDCAAAA Points trace so simple eyes. Short advisers shall not say limitations. Keys stretch in full now blue wings. Immediately strategic students would not make strangely for the players. Books cooking 1.69 12518.00 20.55186637313737424 +AAAAAAAAFDIAAAAA Great pp. will not r Books cooking 1.91 7268.22 11.93285558480304571 +AAAAAAAAGFGCAAAA Products may not resist further specif Books cooking 5.37 2.72 0.00446565557876128 +AAAAAAAAICAAAAAA Somet Books cooking 7.34 580.58 0.95318761614603744 +AAAAAAAAIFHDAAAA Genetic properties might describe therefore leaves; right other organisers must not talk even lives; methods carry thus wrong minutes. Proud worke Books cooking 1.08 1445.15 2.37262579398781566 +AAAAAAAAIHHDAAAA Urgent agencies mean over as a plants; then Books cooking 6.47 1312.79 2.15531911295662354 +AAAAAAAAILAEAAAA As light hundreds must establish on a services. Sometimes special results Books cooking 44.82 3513.30 5.76808372972867366 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q14.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q14.sql.out new file mode 100644 index 0000000000000..46b0ab1a3018d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q14.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +store 1001001 1 1 1217789.46 331 store 1001001 1 1 1234065.50 342 +store 1001002 1 1 553751.85 151 store 1001002 1 1 559183.12 158 +store 1002001 2 1 1249466.39 353 store 1002001 2 1 1576146.08 368 +store 1002002 2 1 547670.08 152 store 1002002 2 1 491380.52 130 +store 1003001 3 1 1167282.59 309 store 1003001 3 1 1045547.02 287 +store 1003002 3 1 789483.23 193 store 1003002 3 1 601149.89 165 +store 1004001 4 1 1522903.35 398 store 1004001 4 1 1062756.03 278 +store 1004002 4 1 541183.58 151 store 1004002 4 1 607217.67 151 +store 2001001 1 2 1145363.89 364 store 2001001 1 2 1303027.45 345 +store 2001002 1 2 627833.76 169 store 2001002 1 2 536535.24 161 +store 2002001 2 2 1440545.64 383 store 2002001 2 2 1329963.82 368 +store 2002002 2 2 747435.75 207 store 2002002 2 2 816248.81 208 +store 2003001 3 2 1388229.40 395 store 2003001 3 2 1469176.64 414 +store 2003002 3 2 716344.82 190 store 2003002 3 2 729626.91 181 +store 2004001 4 2 1613653.10 440 store 2004001 4 2 1488785.35 406 +store 2004002 4 2 657357.03 182 store 2004002 4 2 593298.47 162 +store 3001001 1 3 1282986.35 374 store 3001001 1 3 1519875.76 390 +store 3001002 1 3 673633.15 169 store 3001002 1 3 656845.65 177 +store 3002001 2 3 1333021.08 362 store 3002001 2 3 1241035.46 351 +store 3002002 2 3 748022.41 210 store 3002002 2 3 699267.97 190 +store 3003001 3 3 1134792.91 326 store 3003001 3 3 1020180.77 305 +store 3003002 3 3 802127.48 197 store 3003002 3 3 703596.57 195 +store 3004001 4 3 1385106.21 399 store 3004001 4 3 1338923.03 363 +store 3004002 4 3 732944.48 203 store 3004002 4 3 722668.95 194 +store 4001001 1 4 1157990.42 340 store 4001001 1 4 1349185.21 367 +store 4001002 1 4 621674.47 152 store 4001002 1 4 549243.01 154 +store 4002001 2 4 1568059.38 402 store 4002001 2 4 1452439.59 374 +store 4002002 2 4 694485.13 177 store 4002002 2 4 802206.88 214 +store 4003001 3 4 1593249.87 451 store 4003001 3 4 1456053.57 403 +store 4003002 3 4 662121.36 163 store 4003002 3 4 717341.48 189 +store 4004001 4 4 1248825.41 348 store 4004001 4 4 1389020.06 347 +store 4004002 4 4 718297.35 201 store 4004002 4 4 708937.79 199 +store 5001001 1 5 1273579.39 349 store 5001001 1 5 1253638.00 334 +store 5001002 1 5 825892.06 217 store 5001002 1 5 702168.77 206 +store 5002001 2 5 1486176.43 365 store 5002001 2 5 1243594.81 313 +store 5002002 2 5 633123.05 172 store 5002002 2 5 765400.35 202 +store 5003001 3 5 1477157.87 390 store 5003001 3 5 1314942.13 338 +store 5003002 3 5 697211.48 193 store 5003002 3 5 611056.63 180 +store 5004001 4 5 1287387.74 353 store 5004001 4 5 1459943.10 376 +store 5004002 4 5 587935.16 160 store 5004002 4 5 554584.12 154 +store 6001001 1 6 83017.27 25 store 6001001 1 6 53839.44 19 +store 6001002 1 6 38797.04 9 store 6001002 1 6 40326.43 9 +store 6001003 1 6 50457.02 12 store 6001003 1 6 53585.59 10 +store 6001004 1 6 85147.15 18 store 6001004 1 6 88513.17 27 +store 6001005 1 6 174523.58 45 store 6001005 1 6 96635.15 31 +store 6001006 1 6 61231.20 15 store 6001006 1 6 52077.07 12 +store 6001007 1 6 69263.80 19 store 6001007 1 6 51940.50 18 +store 6001008 1 6 7774.49 5 store 6001008 1 6 34176.00 11 +store 6002001 2 6 137288.11 36 store 6002001 2 6 158530.44 40 +store 6002002 2 6 32548.73 11 store 6002002 2 6 87976.59 21 +store 6002003 2 6 73606.51 23 store 6002003 2 6 75815.97 30 +store 6002004 2 6 53750.06 10 store 6002004 2 6 47235.26 13 +store 6002005 2 6 102178.51 28 store 6002005 2 6 65676.11 26 +store 6002006 2 6 54942.99 11 store 6002006 2 6 19627.43 5 +store 6002007 2 6 90084.17 30 store 6002007 2 6 92767.58 27 +store 6002008 2 6 39639.50 9 store 6002008 2 6 22406.00 6 +store 6003001 3 6 51483.85 13 store 6003001 3 6 54481.52 8 +store 6003003 3 6 47337.51 12 store 6003003 3 6 45051.71 13 +store 6003004 3 6 10107.64 5 store 6003004 3 6 32499.05 7 +store 6003005 3 6 66634.50 24 store 6003005 3 6 108128.10 28 +store 6003006 3 6 48367.44 18 store 6003006 3 6 40436.80 10 +store 6003007 3 6 81724.63 24 store 6003007 3 6 53676.25 15 +store 6003008 3 6 38023.42 10 store 6003008 3 6 86371.33 18 +store 6004001 4 6 46759.68 16 store 6004001 4 6 98037.79 25 +store 6004002 4 6 32304.25 8 store 6004002 4 6 31274.82 9 +store 6004003 4 6 69089.26 18 store 6004003 4 6 70840.13 21 +store 6004004 4 6 40891.73 9 store 6004004 4 6 37496.52 13 +store 6004005 4 6 47341.21 14 store 6004005 4 6 11576.16 4 +store 6004006 4 6 37823.00 15 store 6004006 4 6 95562.26 20 +store 6004007 4 6 107423.33 25 store 6004007 4 6 49625.35 14 +store 6004008 4 6 29725.16 11 store 6004008 4 6 80015.81 19 +store 6005001 5 6 57198.07 20 store 6005001 5 6 61313.19 22 +store 6005002 5 6 26742.25 7 store 6005002 5 6 8143.14 3 +store 6005003 5 6 220162.11 49 store 6005003 5 6 156718.19 37 +store 6005004 5 6 11571.97 5 store 6005004 5 6 47370.39 12 +store 6005005 5 6 98809.34 31 store 6005005 5 6 139898.64 34 +store 6005006 5 6 18108.22 5 store 6005006 5 6 23139.57 5 +store 6005007 5 6 74967.30 18 store 6005007 5 6 49412.83 17 +store 6005008 5 6 71048.66 14 store 6005008 5 6 35086.91 13 +store 6006001 6 6 33781.32 9 store 6006001 6 6 33793.98 13 +store 6006003 6 6 97264.81 19 store 6006003 6 6 68109.24 28 +store 6006004 6 6 68406.20 16 store 6006004 6 6 45670.46 17 +store 6006005 6 6 56934.37 15 store 6006005 6 6 80912.54 19 +store 6006006 6 6 9360.35 4 store 6006006 6 6 63735.59 18 +store 6006007 6 6 15367.23 7 store 6006007 6 6 35007.03 7 +store 6006008 6 6 32684.13 6 store 6006008 6 6 17541.68 5 +store 6007001 7 6 20878.30 5 store 6007001 7 6 31745.91 6 +store 6007002 7 6 22897.56 7 store 6007002 7 6 49259.06 10 +store 6007003 7 6 23239.07 10 store 6007003 7 6 58808.25 19 +store 6007004 7 6 47157.82 10 store 6007004 7 6 35948.03 11 +store 6007005 7 6 108100.32 27 store 6007005 7 6 61513.57 21 +store 6007006 7 6 49256.53 10 store 6007006 7 6 16532.02 5 +store 6007007 7 6 65032.83 21 store 6007007 7 6 110874.54 26 +store 6007008 7 6 56626.18 12 store 6007008 7 6 88630.08 17 +store 6008001 8 6 138363.66 40 store 6008001 8 6 118280.46 34 +store 6008002 8 6 59363.03 20 store 6008002 8 6 35498.23 9 +store 6008004 8 6 48538.79 12 store 6008004 8 6 15279.50 6 +store 6008005 8 6 107128.07 41 store 6008005 8 6 174087.69 43 +store 6008006 8 6 18420.16 7 store 6008006 8 6 19669.85 8 +store 6008007 8 6 33281.27 10 store 6008007 8 6 50246.87 12 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q14a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q14a.sql.out new file mode 100644 index 0000000000000..027c177ab7710 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q14a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL 677178449.86 157050 +catalog NULL NULL NULL 235662013.22 46193 +catalog 1001001 NULL NULL 2553656.44 520 +catalog 1001001 1 NULL 2222983.82 453 +catalog 1001001 1 1 1860468.47 369 +catalog 1001001 1 2 33739.75 3 +catalog 1001001 1 3 49979.66 13 +catalog 1001001 1 4 49610.76 10 +catalog 1001001 1 5 61333.54 13 +catalog 1001001 1 6 31767.56 6 +catalog 1001001 1 7 19752.34 6 +catalog 1001001 1 8 43708.18 12 +catalog 1001001 1 9 36388.85 11 +catalog 1001001 1 10 36234.71 10 +catalog 1001001 2 NULL 41208.06 6 +catalog 1001001 2 3 41208.06 6 +catalog 1001001 3 NULL 85518.68 19 +catalog 1001001 3 1 11946.51 5 +catalog 1001001 3 4 32093.25 7 +catalog 1001001 3 6 6905.04 2 +catalog 1001001 3 7 34573.88 5 +catalog 1001001 4 NULL 22702.37 6 +catalog 1001001 4 7 22702.37 6 +catalog 1001001 5 NULL 15433.39 3 +catalog 1001001 5 9 15433.39 3 +catalog 1001001 8 NULL 22550.38 4 +catalog 1001001 8 9 22550.38 4 +catalog 1001001 10 NULL 29206.10 5 +catalog 1001001 10 7 29206.10 5 +catalog 1001001 13 NULL 9727.21 3 +catalog 1001001 13 9 9727.21 3 +catalog 1001001 14 NULL 80584.37 15 +catalog 1001001 14 9 17298.44 3 +catalog 1001001 14 10 63285.93 12 +catalog 1001001 16 NULL 23742.06 6 +catalog 1001001 16 9 23742.06 6 +catalog 1001002 NULL NULL 2674718.24 515 +catalog 1001002 1 NULL 2413496.48 456 +catalog 1001002 1 1 2413496.48 456 +catalog 1001002 2 NULL 40653.02 10 +catalog 1001002 2 1 40653.02 10 +catalog 1001002 3 NULL 55083.13 11 +catalog 1001002 3 1 55083.13 11 +catalog 1001002 4 NULL 59222.69 15 +catalog 1001002 4 1 59222.69 15 +catalog 1001002 7 NULL 23174.76 3 +catalog 1001002 7 1 23174.76 3 +catalog 1001002 10 NULL 40160.58 11 +catalog 1001002 10 1 40160.58 11 +catalog 1001002 12 NULL 25963.72 6 +catalog 1001002 12 1 25963.72 6 +catalog 1001002 13 NULL 11354.62 2 +catalog 1001002 13 1 11354.62 2 +catalog 1001002 14 NULL 5609.24 1 +catalog 1001002 14 1 5609.24 1 +catalog 1002001 NULL NULL 2907063.96 511 +catalog 1002001 1 NULL 88041.35 10 +catalog 1002001 1 1 38209.40 6 +catalog 1002001 1 5 49831.95 4 +catalog 1002001 2 NULL 2574507.86 463 +catalog 1002001 2 1 2132551.78 377 +catalog 1002001 2 3 34961.36 11 +catalog 1002001 2 4 50761.02 9 +catalog 1002001 2 6 99382.39 16 +catalog 1002001 2 7 43547.86 4 +catalog 1002001 2 8 104272.59 23 +catalog 1002001 2 9 62741.20 11 +catalog 1002001 2 10 46289.66 12 +catalog 1002001 3 NULL 44220.16 5 +catalog 1002001 3 1 44220.16 5 +catalog 1002001 4 NULL 86121.32 15 +catalog 1002001 4 1 28141.23 5 +catalog 1002001 4 3 29119.15 5 +catalog 1002001 4 5 28860.94 5 +catalog 1002001 5 NULL 26371.70 2 +catalog 1002001 5 9 26371.70 2 +catalog 1002001 6 NULL 32428.43 4 +catalog 1002001 6 10 32428.43 4 +catalog 1002001 8 NULL 19630.88 3 +catalog 1002001 8 8 19630.88 3 +catalog 1002001 10 NULL 20942.49 5 +catalog 1002001 10 8 20942.49 5 +catalog 1002001 11 NULL 6069.54 2 +catalog 1002001 11 8 6069.54 2 +catalog 1002001 12 NULL 8730.23 2 +catalog 1002001 12 7 8730.23 2 +catalog 1002002 NULL NULL 2695862.91 567 +catalog 1002002 1 NULL 79482.49 21 +catalog 1002002 1 1 79482.49 21 +catalog 1002002 2 NULL 2153480.10 444 +catalog 1002002 2 1 2153480.10 444 +catalog 1002002 3 NULL 64329.01 20 +catalog 1002002 3 1 64329.01 20 +catalog 1002002 4 NULL 98180.19 26 +catalog 1002002 4 1 98180.19 26 +catalog 1002002 7 NULL 45311.61 6 +catalog 1002002 7 1 45311.61 6 +catalog 1002002 8 NULL 16754.12 4 +catalog 1002002 8 1 16754.12 4 +catalog 1002002 9 NULL 30603.32 7 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q18a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q18a.sql.out new file mode 100644 index 0000000000000..40e6ac10e6a33 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q18a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL 49.988822 101.021588 257.990037 51.117224 -216.279125 1957.948730 3.062849 +AAAAAAAAAAAEAAAA NULL NULL NULL 83.000000 14.310000 0.000000 7.580000 -377.650000 1980.000000 5.000000 +AAAAAAAAAABEAAAA NULL NULL NULL 45.000000 57.980000 0.000000 44.640000 -95.400000 1951.000000 1.000000 +AAAAAAAAAADBAAAA NULL NULL NULL 19.000000 232.670000 0.000000 62.820000 -679.630000 1933.000000 6.000000 +AAAAAAAAAAEDAAAA NULL NULL NULL 57.000000 119.920000 0.000000 119.920000 3329.940000 1946.000000 6.000000 +AAAAAAAAAAFAAAAA NULL NULL NULL 83.000000 141.350000 0.000000 12.720000 -6142.000000 1991.000000 4.000000 +AAAAAAAAAAHBAAAA NULL NULL NULL 97.000000 83.210000 0.000000 71.560000 431.650000 1924.000000 3.000000 +AAAAAAAAAAHDAAAA NULL NULL NULL 53.500000 97.390000 0.000000 52.640000 -6.425000 1974.000000 2.500000 +AAAAAAAAAAKAAAAA NULL NULL NULL 69.000000 127.490000 0.000000 101.990000 2660.640000 1948.000000 2.000000 +AAAAAAAAAALAAAAA NULL NULL NULL 88.000000 143.850000 0.000000 28.770000 -2337.280000 1944.000000 5.000000 +AAAAAAAAAALDAAAA NULL NULL NULL 19.000000 116.590000 0.000000 5.820000 -1366.290000 1949.000000 2.000000 +AAAAAAAAAAMBAAAA NULL NULL NULL 1.000000 37.170000 0.000000 30.100000 13.430000 1932.000000 2.000000 +AAAAAAAAAAOAAAAA NULL NULL NULL 87.000000 26.050000 0.000000 1.820000 -963.960000 1985.000000 4.000000 +AAAAAAAAABAEAAAA NULL NULL NULL 91.000000 67.860000 56.180000 8.820000 -4041.070000 1970.000000 0.000000 +AAAAAAAAABBDAAAA NULL NULL NULL 71.000000 31.880000 0.000000 15.300000 -23.430000 1979.000000 6.000000 +AAAAAAAAABCEAAAA NULL NULL NULL 65.000000 115.550000 0.000000 25.420000 -1641.900000 1973.000000 6.000000 +AAAAAAAAABDAAAAA NULL NULL NULL 71.000000 40.670000 0.000000 13.820000 -269.090000 1968.000000 0.000000 +AAAAAAAAABFCAAAA NULL NULL NULL 2.000000 237.540000 0.000000 35.630000 -101.500000 1932.000000 1.000000 +AAAAAAAAABGBAAAA NULL NULL NULL 25.500000 149.375000 0.000000 87.680000 -85.130000 1955.000000 4.000000 +AAAAAAAAABGEAAAA NULL NULL NULL 32.500000 237.875000 0.000000 107.305000 615.725000 1969.000000 3.000000 +AAAAAAAAABJDAAAA NULL NULL NULL 12.000000 118.140000 0.000000 54.340000 -405.960000 1971.000000 4.000000 +AAAAAAAAACAAAAAA NULL NULL NULL 40.000000 218.900000 0.000000 168.550000 3400.000000 1984.000000 2.000000 +AAAAAAAAACDCAAAA NULL NULL NULL 9.000000 27.400000 0.000000 18.630000 -11.070000 1982.000000 0.000000 +AAAAAAAAACDDAAAA NULL NULL NULL 9.000000 72.980000 0.000000 36.490000 85.140000 1925.000000 4.000000 +AAAAAAAAACGDAAAA NULL NULL NULL 100.000000 240.000000 0.000000 52.800000 -3171.000000 1984.000000 2.000000 +AAAAAAAAACJAAAAA NULL NULL NULL 89.000000 141.410000 0.000000 91.910000 3394.460000 1939.000000 5.000000 +AAAAAAAAACJCAAAA NULL NULL NULL 97.000000 68.650000 0.000000 60.410000 2160.190000 1979.000000 5.000000 +AAAAAAAAACMAAAAA NULL NULL NULL 36.000000 5.030000 0.000000 1.200000 -23.400000 1973.000000 6.000000 +AAAAAAAAACPDAAAA NULL NULL NULL 89.000000 165.880000 0.000000 48.100000 -1248.670000 1988.000000 5.000000 +AAAAAAAAADDEAAAA NULL NULL NULL 44.000000 152.720000 0.000000 138.970000 3326.400000 1930.000000 6.000000 +AAAAAAAAADFAAAAA NULL NULL NULL 52.000000 99.460000 465.420000 59.670000 242.820000 1945.000000 2.000000 +AAAAAAAAADGBAAAA NULL NULL NULL 51.000000 48.560000 0.000000 12.620000 -1473.390000 1935.000000 3.000000 +AAAAAAAAADIAAAAA NULL NULL NULL 27.000000 175.020000 0.000000 47.250000 -388.260000 1924.000000 3.000000 +AAAAAAAAADMCAAAA NULL NULL NULL 89.000000 161.440000 0.000000 35.510000 -4648.470000 1924.000000 6.000000 +AAAAAAAAADNAAAAA NULL NULL NULL 57.000000 107.880000 975.770000 24.810000 -2503.940000 1975.000000 2.000000 +AAAAAAAAADNBAAAA NULL NULL NULL 32.000000 52.410000 0.000000 33.540000 274.560000 1975.000000 5.000000 +AAAAAAAAADPBAAAA NULL NULL NULL 72.000000 65.940000 3.770000 1.310000 -3562.010000 1932.000000 5.000000 +AAAAAAAAAEABAAAA NULL NULL NULL 57.000000 200.610000 0.000000 24.070000 -4047.570000 1987.000000 5.000000 +AAAAAAAAAEADAAAA NULL NULL NULL 61.000000 28.010000 0.000000 18.480000 85.400000 1970.000000 2.000000 +AAAAAAAAAEAEAAAA NULL NULL NULL 95.000000 155.800000 0.000000 4.670000 -5184.150000 1966.000000 4.000000 +AAAAAAAAAEBDAAAA NULL NULL NULL 57.000000 257.970000 0.000000 36.110000 -2994.780000 1990.000000 4.000000 +AAAAAAAAAECCAAAA NULL NULL NULL 40.000000 81.220000 0.000000 51.160000 746.800000 1975.000000 1.000000 +AAAAAAAAAEDAAAAA NULL NULL NULL 63.000000 31.950000 564.530000 14.690000 -809.600000 1966.000000 6.000000 +AAAAAAAAAEJAAAAA NULL NULL NULL 83.000000 48.220000 202.630000 6.260000 -2670.220000 1944.000000 3.000000 +AAAAAAAAAEJDAAAA NULL NULL NULL 4.000000 39.860000 0.000000 0.790000 -66.480000 1958.000000 3.000000 +AAAAAAAAAELCAAAA NULL NULL NULL 34.000000 106.700000 0.000000 17.070000 -1481.040000 1925.000000 1.000000 +AAAAAAAAAENAAAAA NULL NULL NULL 49.000000 205.770000 0.000000 141.980000 3018.400000 1926.000000 2.000000 +AAAAAAAAAENCAAAA NULL NULL NULL 67.000000 121.280000 0.000000 115.210000 4025.360000 1947.000000 3.000000 +AAAAAAAAAEPAAAAA NULL NULL NULL 73.000000 85.250000 0.000000 3.410000 -4651.560000 1975.000000 1.000000 +AAAAAAAAAEPDAAAA NULL NULL NULL 28.000000 91.090000 0.000000 43.720000 -177.240000 1987.000000 5.000000 +AAAAAAAAAFADAAAA NULL NULL NULL 15.000000 109.790000 0.000000 62.580000 139.200000 1963.000000 4.000000 +AAAAAAAAAFCAAAAA NULL NULL NULL 23.000000 8.880000 0.000000 1.950000 -67.390000 1988.000000 0.000000 +AAAAAAAAAFCBAAAA NULL NULL NULL 74.000000 14.690000 0.000000 10.720000 -69.560000 1954.000000 3.000000 +AAAAAAAAAFHCAAAA NULL NULL NULL 53.000000 48.210000 350.520000 47.240000 5.640000 1960.000000 1.000000 +AAAAAAAAAFMCAAAA NULL NULL NULL 42.000000 56.730000 828.590000 53.320000 -380.870000 1944.000000 5.000000 +AAAAAAAAAFODAAAA NULL NULL NULL 15.000000 23.690000 0.000000 17.530000 139.950000 1973.000000 5.000000 +AAAAAAAAAFPAAAAA NULL NULL NULL 89.000000 126.010000 0.000000 75.600000 -507.300000 1938.000000 6.000000 +AAAAAAAAAGEBAAAA NULL NULL NULL 55.000000 78.640000 0.000000 7.070000 -1731.400000 1957.000000 0.000000 +AAAAAAAAAGEDAAAA NULL NULL NULL 19.000000 205.240000 0.000000 22.570000 -1223.600000 1935.000000 0.000000 +AAAAAAAAAGEEAAAA NULL NULL NULL 40.000000 128.550000 488.400000 48.840000 -882.800000 1934.000000 6.000000 +AAAAAAAAAGIDAAAA NULL NULL NULL 11.000000 125.030000 0.000000 13.750000 -753.610000 1935.000000 0.000000 +AAAAAAAAAGJCAAAA NULL NULL NULL 8.000000 190.590000 0.000000 142.940000 369.520000 1941.000000 4.000000 +AAAAAAAAAGKDAAAA NULL NULL NULL 80.000000 257.180000 0.000000 100.300000 953.600000 1960.000000 1.000000 +AAAAAAAAAGLAAAAA NULL NULL NULL 44.000000 58.460000 411.400000 9.350000 -1811.480000 1973.000000 6.000000 +AAAAAAAAAGMBAAAA NULL NULL NULL 86.000000 110.960000 0.000000 4.430000 -3732.400000 1946.000000 0.000000 +AAAAAAAAAGNBAAAA NULL NULL NULL 41.000000 133.900000 0.000000 1.330000 -3118.870000 1948.000000 5.000000 +AAAAAAAAAGPBAAAA NULL NULL NULL 45.000000 57.940000 876.960000 33.600000 -529.110000 1965.000000 6.000000 +AAAAAAAAAHAAAAAA NULL NULL NULL 20.000000 51.470000 0.000000 26.240000 -205.400000 1942.000000 5.000000 +AAAAAAAAAHBAAAAA NULL NULL NULL 50.500000 70.085000 0.000000 55.760000 724.565000 1957.000000 2.000000 +AAAAAAAAAHDEAAAA NULL NULL NULL 79.000000 21.510000 0.000000 7.950000 -631.210000 1943.000000 6.000000 +AAAAAAAAAHEDAAAA NULL NULL NULL 93.000000 23.200000 0.000000 2.320000 -1272.240000 1948.000000 5.000000 +AAAAAAAAAHFEAAAA NULL NULL NULL 75.000000 57.750000 508.650000 14.430000 -870.150000 1965.000000 6.000000 +AAAAAAAAAHGDAAAA NULL NULL NULL 71.000000 3.020000 0.000000 1.540000 -80.940000 1984.000000 2.000000 +AAAAAAAAAHIBAAAA NULL NULL NULL 30.500000 59.105000 0.000000 41.310000 452.755000 1957.500000 3.000000 +AAAAAAAAAHKAAAAA NULL NULL NULL 38.000000 89.720000 0.000000 20.630000 -1257.800000 1939.000000 5.000000 +AAAAAAAAAHKCAAAA NULL NULL NULL 71.000000 141.010000 0.000000 90.240000 2269.870000 1924.000000 0.000000 +AAAAAAAAAHNDAAAA NULL NULL NULL 51.000000 63.060000 0.000000 5.040000 -2107.830000 1975.000000 1.000000 +AAAAAAAAAIDAAAAA NULL NULL NULL 77.000000 36.770000 0.000000 26.100000 -72.380000 1986.000000 2.000000 +AAAAAAAAAIEEAAAA NULL NULL NULL 1.000000 145.160000 0.000000 133.540000 45.560000 1958.000000 3.000000 +AAAAAAAAAIIAAAAA NULL NULL NULL 53.000000 232.610000 0.000000 0.000000 -4466.840000 1971.000000 4.000000 +AAAAAAAAAIJAAAAA NULL NULL NULL 91.000000 62.950000 0.000000 26.430000 -107.380000 1935.000000 6.000000 +AAAAAAAAAIKBAAAA NULL NULL NULL 73.000000 6.890000 124.470000 2.890000 -280.690000 1935.000000 3.000000 +AAAAAAAAAILBAAAA NULL NULL NULL 7.000000 15.340000 0.000000 1.070000 -73.290000 1966.000000 4.000000 +AAAAAAAAAILDAAAA NULL NULL NULL 27.500000 69.010000 200.210000 38.180000 -143.600000 1974.000000 2.500000 +AAAAAAAAAIODAAAA NULL NULL NULL 99.000000 84.610000 663.300000 60.910000 -931.590000 1986.000000 1.000000 +AAAAAAAAAIPCAAAA NULL NULL NULL 96.000000 183.920000 0.000000 22.070000 -3887.040000 1924.000000 3.000000 +AAAAAAAAAJACAAAA NULL NULL NULL 62.000000 128.680000 0.000000 104.230000 2698.860000 1975.000000 4.000000 +AAAAAAAAAJAEAAAA NULL NULL NULL 81.000000 131.730000 0.000000 23.710000 -4963.680000 1968.000000 5.000000 +AAAAAAAAAJBAAAAA NULL NULL NULL 19.000000 113.950000 0.000000 79.760000 613.320000 1930.000000 2.000000 +AAAAAAAAAJBBAAAA NULL NULL NULL 60.500000 140.315000 1093.800000 51.610000 -1455.575000 1976.000000 2.500000 +AAAAAAAAAJBDAAAA NULL NULL NULL 4.000000 85.990000 82.520000 20.630000 -252.920000 1988.000000 5.000000 +AAAAAAAAAJDEAAAA NULL NULL NULL 21.000000 7.750000 38.220000 7.280000 42.000000 1948.000000 0.000000 +AAAAAAAAAJEAAAAA NULL NULL NULL 47.000000 95.750000 0.000000 44.040000 -1744.170000 1935.000000 0.000000 +AAAAAAAAAJEDAAAA NULL NULL NULL 27.000000 38.270000 0.000000 21.040000 -300.240000 1976.000000 6.000000 +AAAAAAAAAJFDAAAA NULL NULL NULL 82.000000 22.310000 0.000000 6.690000 -80.360000 1973.000000 5.000000 +AAAAAAAAAJGBAAAA NULL NULL NULL 64.000000 155.160000 0.000000 34.130000 -3623.040000 1975.000000 5.000000 +AAAAAAAAAJGCAAAA NULL NULL NULL 8.000000 2.080000 0.000000 1.200000 0.720000 1932.000000 0.000000 +AAAAAAAAAJJCAAAA NULL NULL NULL 61.000000 30.960000 0.000000 0.610000 -875.350000 1932.000000 2.000000 +AAAAAAAAAJKBAAAA NULL NULL NULL 13.000000 30.910000 0.000000 14.520000 43.680000 1928.000000 6.000000 +AAAAAAAAAJLCAAAA NULL NULL NULL 25.000000 7.330000 0.000000 0.580000 -116.500000 1970.000000 0.000000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q20.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q20.sql.out new file mode 100644 index 0000000000000..49c6cf986ae4c --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q20.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAMJJBAAAA NULL Books NULL NULL 9010.49 72.07596281370536693 +AAAAAAAAMLGDAAAA NULL Books NULL 6.35 1491.96 11.93436244638591899 +AAAAAAAAAELBAAAA Precisely elderly bodies Books arts 1.40 4094.31 1.52075020571659240 +AAAAAAAAAFODAAAA Close, precise teeth should go for a qualities. Political groups shall not become just important occasions. Trials mean ne Books arts 2.53 332.38 0.12345595555199313 +AAAAAAAAAHMAAAAA Abilities could affect cruel parts. Predominantly other events telephone strong signs. Accurate mate Books arts 25.69 2626.56 0.97558359291967949 +AAAAAAAAAJOAAAAA Average parents require also times. Children would not describe lightly purposes; large miles love now correct relations. Usual, german goals proceed literary, wooden visitors. Initial councils wil Books arts 1.24 12327.20 4.57869383019594946 +AAAAAAAAALNCAAAA Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 343.80 0.12769768794384511 +AAAAAAAAANKAAAAA Small objects stop etc mediterranean patterns; liberal, free initiatives would not leave less clear british attitudes; good, blue relationships find softly very Books arts 58.41 886.92 0.32942883476194038 +AAAAAAAABGDAAAAA Newly national rights head curiously all electrical cells. Chinese, long values might not pull bad lines. High fun clothes ough Books arts 3.28 2219.85 0.82451923380495801 +AAAAAAAACBBAAAAA Quick, easy studies must make always necessary systems. Upper, new persons should buy much physical technologies. English sciences hear solicitors. S Books arts 0.99 2050.16 0.76149125047979491 +AAAAAAAACMFDAAAA Early, short v Books arts 75.57 5429.86 2.01681375177070042 +AAAAAAAADLLDAAAA Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 13539.35 5.02892289488801418 +AAAAAAAAEIPCAAAA Scientific, difficult polls would not achieve. Countries reach of course. Bad, new churches realize most english Books arts 3.98 143.88 0.05344137097545211 +AAAAAAAAFCIBAAAA United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 12297.15 4.56753235398096242 +AAAAAAAAFFIBAAAA Naturally new years put serious, negative vehicles. Fin Books arts 3.34 4587.47 1.70392470189572752 +AAAAAAAAFJGCAAAA Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Books arts 4.03 5359.20 1.99056849688381241 +AAAAAAAAFLNCAAAA Much new waters Books arts 1.85 6718.63 2.49550179508480530 +AAAAAAAAGHBAAAAA Hard different differences would not paint even. Together suitable schemes marry directly only open women. Social ca Books arts 2.65 3208.60 1.19177080144450674 +AAAAAAAAGLEBAAAA Tall, following actions keep widely willing, secondary groups. Heads could afford however; agricultural, square pri Books arts 9.99 4780.52 1.77562929368618505 +AAAAAAAAGMFAAAAA Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 5622.46 2.08835119999055082 +AAAAAAAAHHEBAAAA Powerful walls will find; there scottish decades must not Books arts 4.16 7914.41 2.93965054810833964 +AAAAAAAAHMCEAAAA Too executive doors progress mainly seemingly possible parts; hundreds stay virtually simple workers. Sola Books arts 34.32 3029.48 1.12524023173973205 +AAAAAAAAIBOCAAAA Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 1450.99 0.53894144336718969 +AAAAAAAAICMBAAAA Aside legitimate decisions may not stand probably sexual g Books arts 3.88 9619.83 3.57309496629679899 +AAAAAAAAIFPBAAAA Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 13616.57 5.05760473160419719 +AAAAAAAAIHNAAAAA Unlikely states take later in general extra inf Books arts 0.32 11879.56 4.41242683475911751 +AAAAAAAAINFDAAAA Sometimes careful things state probably so Books arts 5.08 25457.85 9.45581321995700176 +AAAAAAAAJGHDAAAA Circumstances would not use. Principles seem writers. Times go from a hands. Members find grounds. Central, only teachers pursue properly into a p Books arts 5.95 2567.54 0.95366178505916251 +AAAAAAAAJLHBAAAA Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 1811.85 0.67297572978782944 +AAAAAAAAKFGBAAAA Systems cannot await regions. Home appropr Books arts 7.30 1730.16 0.64263360027028230 +AAAAAAAAKHLBAAAA Extra, primitive weeks look obviou Books arts 1.18 22.77 0.00845746467272063 +AAAAAAAALCFBAAAA More than key reasons should remain. Words used to offer slowly british Books arts 0.28 10311.18 3.82988320527288194 +AAAAAAAALGEEAAAA Children may turn also above, historical aspects. Surveys migh Books arts 7.22 11872.32 4.40973768042312729 +AAAAAAAALOKCAAAA Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 8348.99 3.10106666569599586 +AAAAAAAAMACDAAAA Useful observers start often white colleagues; simple pro Books arts 3.47 7565.51 2.81005856636428042 +AAAAAAAAMNPAAAAA Members should say earnings. Detailed departments would not move just at the hopes. Figures can take. Actually open houses want. Good teachers combine the Books arts 3.09 4363.97 1.62091006182752106 +AAAAAAAAMPFCAAAA Major, senior words afford economic libraries; successful seconds need outside. Clinical, new ideas put now red c Books arts 5.87 9661.08 3.58841646026911898 +AAAAAAAANABCAAAA Likely states feel astonishingly working roads. Parents put so somewhere able policies. Others may rely shortly instead interesting bodies; bri Books arts 7.50 132.66 0.04927392461498107 +AAAAAAAANMECAAAA Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 20114.53 7.47114303396483641 +AAAAAAAAOAHCAAAA Accurate years want then other organisations. Simple lines mean as well so red results. Orthodox, central scales will not in Books arts 7.69 2153.04 0.79970398502215321 +AAAAAAAAODBEAAAA Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 231.71 0.08606408165639427 +AAAAAAAAOKEDAAAA Visitors could not allow glad wages. Communist, real figures used to apply factors. Aggressive, optimistic days must mean about trees. Detailed courts consider really large pro Books arts 9.08 24425.09 9.07221501111207600 +AAAAAAAAOODBAAAA Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 4380.23 1.62694951617879192 +AAAAAAAAAJJBAAAA New, reluctant associations see more different, physical symptoms; useful pounds ought to give. Subjects Books business 9.02 3044.02 1.58609001939612781 +AAAAAAAABDMAAAAA Imports involve most now indian women. Developments announce intimately in a copies. Projects Books business 3.26 472.29 0.24608723177265498 +AAAAAAAABINDAAAA Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 30227.05 15.74983814849696292 +AAAAAAAACAADAAAA Whole, important problems make. Indeed industrial members go skills. Soft Books business 3.22 137.92 0.07186336997625310 +AAAAAAAACPBBAAAA Other, black houses flow. New soldiers put only eastern hours. Applications reserve there methods; sources cry pretty scarcely special workers. Never british opportunities Books business 8.20 736.96 0.38399383075478162 +AAAAAAAAEBPAAAAA Rows could not Books business 1.65 1290.88 0.67261446516056841 +AAAAAAAAEEFDAAAA Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 1065.30 0.55507575431918810 +AAAAAAAAEFEEAAAA White members see highly on a negotiations. Evident, passive colours can refer familiar, ugly factors; away small examinations shall prove Books business 17.97 1446.00 0.75343991433919646 +AAAAAAAAEGCCAAAA Manufacturing, ready concerns see already then new pupils. Both stable types used to manage otherw Books business 1.18 2635.71 1.37333963805184198 +AAAAAAAAFCGDAAAA Small, capable centres Books business 2.98 5029.45 2.62060053746422658 +AAAAAAAAFDLAAAAA Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 621.26 0.32370821658531756 +AAAAAAAAFEGEAAAA Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 3950.22 2.05826654109334761 +AAAAAAAAGIJAAAAA Always other hours used to use. Women should jump then. Civil samples take therefore other offices. Concrete, major demands Books business 1.42 2013.79 1.04928752772968910 +AAAAAAAAHDKCAAAA Visual fragments Books business 6.77 930.13 0.48464527491308216 +AAAAAAAAHDLBAAAA Classic issues will draw as european, engl Books business 75.64 556.83 0.29013689315456070 +AAAAAAAAHJAAAAAA Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 1898.13 0.98902275560488173 +AAAAAAAAHLKAAAAA Confident, video-tape Books business 3.17 1131.00 0.58930881266779474 +AAAAAAAAIHNDAAAA Of course fundamental children will not deal still including a suppliers. More crucial powers will not keep enough. As good comments used to devote even convenient electric problems. Publi Books business 8.85 414.75 0.21610595053401226 +AAAAAAAAIMJAAAAA Departments could seek now for a commu Books business 5.93 9895.85 5.15624369039663714 +AAAAAAAAJFBEAAAA Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 13809.44 7.19542412909562460 +AAAAAAAAJJGBAAAA Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 9267.25 4.82871095861681771 +AAAAAAAAKBMDAAAA So white countries could secure more angry items. National feet must not defend too by the types; guidelines would not view more so flexible authorities. Critics will handle closely lig Books business 2.50 2542.50 1.32477246349059959 +AAAAAAAAKJHDAAAA Simple changes ought to vote almost sudden techniques. Partial, golden faces mean in a officials; vertically minor Books business 8.73 22710.22 11.83318548507904997 +AAAAAAAAKJOBAAAA Christian lines stand once deep formal aspirations. National, fine islands play together with a patterns. New journals lose etc positive armie Books business 4.89 11560.78 6.02375732565303988 +AAAAAAAAKKDAAAAA Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 9065.09 4.72337526492192700 +AAAAAAAAKLCCAAAA Lips will n Books business 8.48 541.62 0.28221170567385587 +AAAAAAAAKNJCAAAA White fees might combine reports. Tr Books business 2.09 37.60 0.01959152197728478 +AAAAAAAALAJCAAAA Asleep children invite more. Wealthy forms could expect as. Indeed statistical examinations could la Books business 3.71 2082.24 1.08495347664844290 +AAAAAAAALDHBAAAA Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 13412.42 6.98855641485568838 +AAAAAAAALHMBAAAA Local, bloody names Books business 4.40 1997.44 1.04076834197626873 +AAAAAAAALJJCAAAA Large, larg Books business 3.50 12097.82 6.30358261721370521 +AAAAAAAANJLBAAAA Only, gothic Books business 1.68 5708.95 2.97465477106967886 +AAAAAAAANKCAAAAA Low, large clouds will not visit for example as the notions. Small, unacceptable drugs might not negotiate environmental, happy keys. Books business 3.11 3020.85 1.57401726502874248 +AAAAAAAAOAPAAAAA Silver, critical operations could help howev Books business 5.56 2286.24 1.19124790439754116 +AAAAAAAAOBAEAAAA Terrible, psychiatric bones will destroy also used studies; solely usual windows should not make shares. Advances continue sufficiently. As key days might not use far artists. Offici Books business 5.83 6672.40 3.47666146918178041 +AAAAAAAAOCHCAAAA Too white addresses end by the talks. Hands get only companies. Statements know. Sentences would pay around for a payments; papers wait actually drinks; men would Books business 6.06 7609.35 3.96486031270882752 +AAAAAAAAAGLDAAAA New, big arguments may not win since by a tenant Books computers 1.00 904.16 0.32327741862037314 +AAAAAAAAALNBAAAA Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 18752.88 6.70498876094676063 +AAAAAAAABEBEAAAA Cheap, desirable members take immediate, estimated debts; months must track typica Books computers 3.26 10027.86 3.58540600677589698 +AAAAAAAABHOAAAAA Expert, scottish terms will ask quiet demands; poor bits attempt northern, dangerous si Books computers 2.66 7330.68 2.62104418148557444 +AAAAAAAACCDBAAAA Gradually serious visitors bear no doubt technical hearts. Critics continue earlier soviet, standard minute Books computers 6.40 1711.84 0.61205894564136830 +AAAAAAAACCPBAAAA Clear, general goods must know never women. Communications meet about. Other rewards spot wide in a skills. Relative, empty drawings facilitate too rooms. Still asian police end speedily comp Books computers 7.64 1292.04 0.46196177220211789 +AAAAAAAACFMBAAAA At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 3033.10 1.08446816760026298 +AAAAAAAACLPDAAAA Environmental, new women pay again fingers. Different, uncomfortable records miss far russian, dependent members. Enough double men will go here immediatel Books computers 89.89 8553.39 3.05821739476786568 +AAAAAAAACOFCAAAA Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 3411.40 1.21972724504682903 +AAAAAAAACONDAAAA Magnetic Books computers 57.19 3569.09 1.27610843437421206 +AAAAAAAADAHAAAAA Ga Books computers 5.53 2687.70 0.96097230360331899 +AAAAAAAADDBAAAAA S Books computers 65.78 1613.04 0.57673355084432699 +AAAAAAAAEAHCAAAA Simple year Books computers 3.01 1262.79 0.45150359611088856 +AAAAAAAAECEEAAAA Agricultural players shall smoke. So full reasons undertake Books computers 0.70 4408.27 1.57615261257037727 +AAAAAAAAECGEAAAA Then basic years can encourage later traditions. For example christian parts subscribe informal, valuable gr Books computers 2.75 844.19 0.30183547604973987 +AAAAAAAAECHAAAAA Boxes batt Books computers 0.83 15300.82 5.47072375727191844 +AAAAAAAAEIGCAAAA Separate, dead buildings think possibly english, net policies. Big divisions can use almost Books computers 9.46 12403.71 4.43487806374503246 +AAAAAAAAEJECAAAA Artists make times. Rather ready functions must pre Books computers 5.71 1533.00 0.54811569052494252 +AAAAAAAAEMKDAAAA Advantages emerge moves; special, expected operations pass etc natural preferences; very posit Books computers 0.15 5241.45 1.87405152387603389 +AAAAAAAAFGLAAAAA Since other birds shall blame sudden Books computers 6.74 2098.16 0.75018552983158082 +AAAAAAAAFHNAAAAA Legs throw then. Old-fashioned develo Books computers 2.66 163.26 0.05837271209073850 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q22.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q22.sql.out new file mode 100644 index 0000000000000..e8a553dd5094e --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q22.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +esepriableanti NULL NULL NULL 429.7808764940239 +esepriableanti importoamalg #x NULL NULL 429.7808764940239 +esepriableanti importoamalg #x fragrances NULL 429.7808764940239 +esepriableanti importoamalg #x fragrances Women 429.7808764940239 +n stbarn stbarought NULL NULL NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x NULL NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x accessories NULL 430.0122448979592 +n stbarn stbarought amalgimporto #x accessories Men 430.0122448979592 +antiationeing NULL NULL NULL 437.03614457831327 +antiationeing amalgexporti #x NULL NULL 437.03614457831327 +antiationeing amalgexporti #x newborn NULL 437.03614457831327 +antiationeing amalgexporti #x newborn Children 437.03614457831327 +n stpriantin st NULL NULL NULL 438.77868852459017 +n stpriantin st exportiexporti #x NULL NULL 438.77868852459017 +n stpriantin st exportiexporti #x toddlers NULL 438.77868852459017 +n stpriantin st exportiexporti #x toddlers Children 438.77868852459017 +eingprically NULL NULL NULL 439.97975708502025 +eingprically amalgbrand #x NULL NULL 439.97975708502025 +eingprically amalgbrand #x semi-precious NULL 439.97975708502025 +eingprically amalgbrand #x semi-precious Jewelry 439.97975708502025 +prieingable NULL NULL NULL 440.096 +prieingable exportiunivamalg #x NULL NULL 440.096 +prieingable exportiunivamalg #x self-help NULL 440.096 +prieingable exportiunivamalg #x self-help Books 440.096 +oughteingn stationought NULL NULL NULL 440.1497975708502 +oughteingn stationought amalgscholar #x NULL NULL 440.1497975708502 +oughteingn stationought amalgscholar #x rock NULL 440.1497975708502 +oughteingn stationought amalgscholar #x rock Music 440.1497975708502 +eingationbaroughtought NULL NULL NULL 440.9721115537849 +eingationbaroughtought maxicorp #x NULL NULL 440.9721115537849 +eingationbaroughtought maxicorp #x womens watch NULL 440.9721115537849 +eingationbaroughtought maxicorp #x womens watch Jewelry 440.9721115537849 +priantibarpri NULL NULL NULL 443.45849802371544 +priantibarpri exportiimporto #x NULL NULL 443.45849802371544 +priantibarpri exportiimporto #x pants NULL 443.45849802371544 +priantibarpri exportiimporto #x pants Men 443.45849802371544 +prioughtantiation NULL NULL NULL 443.8825910931174 +prioughtantiation corpmaxi #x NULL NULL 443.8825910931174 +prioughtantiation corpmaxi #x parenting NULL 443.8825910931174 +prioughtantiation corpmaxi #x parenting Books 443.8825910931174 +eseprieingoughtought NULL NULL NULL 445.2016129032258 +eseprieingoughtought importonameless #x NULL NULL 445.2016129032258 +eseprieingoughtought importonameless #x baseball NULL 445.2016129032258 +eseprieingoughtought importonameless #x baseball Sports 445.2016129032258 +eingpriationanti NULL NULL NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x NULL NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x fiction NULL 445.4920634920635 +eingpriationanti scholarunivamalg #x fiction Books 445.4920634920635 +antin stablecallyought NULL NULL NULL 445.54918032786884 +antin stablecallyought importoedu pack #x NULL NULL 445.54918032786884 +antin stablecallyought importoedu pack #x mens NULL 445.54918032786884 +antin stablecallyought importoedu pack #x mens Shoes 445.54918032786884 +callycallyn steing NULL NULL NULL 445.9012345679012 +callycallyn steing corpunivamalg #x NULL NULL 445.9012345679012 +callycallyn steing corpunivamalg #x mystery NULL 445.9012345679012 +callycallyn steing corpunivamalg #x mystery Books 445.9012345679012 +oughtpribarought NULL NULL NULL 446.125 +oughtpribarought exportinameless #x NULL NULL 446.125 +oughtpribarought exportinameless #x wallpaper NULL 446.125 +oughtpribarought exportinameless #x wallpaper Home 446.125 +oughtantioughtbarought NULL NULL NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x NULL NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x entertainments NULL 446.1847389558233 +oughtantioughtbarought edu packmaxi #x entertainments Books 446.1847389558233 +callyoughtn stcallyought NULL NULL NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x NULL NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x pop NULL 446.43650793650795 +callyoughtn stcallyought exportischolar #x pop Music 446.43650793650795 +ationeingationableought NULL NULL NULL 446.48192771084337 +ationeingationableought namelessnameless #x NULL NULL 446.48192771084337 +ationeingationableought namelessnameless #x outdoor NULL 446.48192771084337 +ationeingationableought namelessnameless #x outdoor Sports 446.48192771084337 +priantiableese NULL NULL NULL 446.85483870967744 +priantiableese exportiedu pack #x NULL NULL 446.85483870967744 +priantiableese exportiedu pack #x kids NULL 446.85483870967744 +priantiableese exportiedu pack #x kids Shoes 446.85483870967744 +prieseeseableought NULL NULL NULL 446.9186991869919 +prieseeseableought amalgscholar #x NULL NULL 446.9186991869919 +prieseeseableought amalgscholar #x rock NULL 446.9186991869919 +prieseeseableought amalgscholar #x rock Music 446.9186991869919 +ationableoughtcallyought NULL NULL NULL 447.165991902834 +ationableoughtcallyought exportischolar #x NULL NULL 447.165991902834 +ationableoughtcallyought exportischolar #x pop NULL 447.165991902834 +ationableoughtcallyought exportischolar #x pop Music 447.165991902834 +pripricallyese NULL NULL NULL 447.2550607287449 +pripricallyese edu packimporto #x NULL NULL 447.2550607287449 +pripricallyese edu packimporto #x sports-apparel NULL 447.2550607287449 +pripricallyese edu packimporto #x sports-apparel Men 447.2550607287449 +eingableationn st NULL NULL NULL 447.3541666666667 +eingableationn st namelessmaxi #x NULL NULL 447.3541666666667 +eingableationn st namelessmaxi #x romance NULL 447.3541666666667 +eingableationn st namelessmaxi #x romance Books 447.3541666666667 +n stantin stoughtought NULL NULL NULL 448.2396694214876 +n stantin stoughtought importoscholar #x NULL NULL 448.2396694214876 +n stantin stoughtought importoscholar #x country NULL 448.2396694214876 +n stantin stoughtought importoscholar #x country Music 448.2396694214876 +n steingbaranti NULL NULL NULL 448.702479338843 +n steingbaranti amalgamalg #x NULL NULL 448.702479338843 +n steingbaranti amalgamalg #x dresses NULL 448.702479338843 +n steingbaranti amalgamalg #x dresses Women 448.702479338843 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q22a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q22a.sql.out new file mode 100644 index 0000000000000..42f0ab7f6e1f0 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q22a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +oughtn steingcally NULL NULL NULL 429.75206611570246 +oughtn steingcally exportiexporti #x NULL NULL 429.75206611570246 +oughtn steingcally exportiexporti #x toddlers NULL 429.75206611570246 +oughtn steingcally exportiexporti #x toddlers Children 429.75206611570246 +oughtcallypripriought NULL NULL NULL 433.04918032786884 +oughtcallypripriought corpunivamalg #x NULL NULL 433.04918032786884 +oughtcallypripriought corpunivamalg #x musical NULL 433.04918032786884 +oughtcallypripriought corpunivamalg #x musical Electronics 433.04918032786884 +oughtesecallyeseought NULL NULL NULL 434.8091286307054 +oughtesecallyeseought scholarbrand #x NULL NULL 434.8091286307054 +oughtesecallyeseought scholarbrand #x blinds/shades NULL 434.8091286307054 +oughtesecallyeseought scholarbrand #x blinds/shades Home 434.8091286307054 +antiesen stese NULL NULL NULL 436.39676113360326 +antiesen stese edu packexporti #x NULL NULL 436.39676113360326 +antiesen stese edu packexporti #x school-uniforms NULL 436.39676113360326 +antiesen stese edu packexporti #x school-uniforms Children 436.39676113360326 +priesecallyantiought NULL NULL NULL 440.51612903225805 +priesecallyantiought importounivamalg #x NULL NULL 440.51612903225805 +priesecallyantiought importounivamalg #x home repair NULL 440.51612903225805 +priesecallyantiought importounivamalg #x home repair Books 440.51612903225805 +prioughtableoughtought NULL NULL NULL 440.97478991596637 +prioughtableoughtought exportiunivamalg #x NULL NULL 440.97478991596637 +prioughtableoughtought exportiunivamalg #x dvd/vcr players NULL 440.97478991596637 +prioughtableoughtought exportiunivamalg #x dvd/vcr players Electronics 440.97478991596637 +antiablen stought NULL NULL NULL 441.520325203252 +antiablen stought amalgamalg #x NULL NULL 441.520325203252 +antiablen stought amalgamalg #x dresses NULL 441.520325203252 +antiablen stought amalgamalg #x dresses Women 441.520325203252 +n steingantianti NULL NULL NULL 442.4404761904762 +n steingantianti corpbrand #x NULL NULL 442.4404761904762 +n steingantianti corpbrand #x rugs NULL 442.4404761904762 +n steingantianti corpbrand #x rugs Home 442.4404761904762 +prieingation NULL NULL NULL 442.68595041322317 +prieingation importoamalg #x NULL NULL 442.68595041322317 +prieingation importoamalg #x fragrances NULL 442.68595041322317 +prieingation importoamalg #x fragrances Women 442.68595041322317 +n stesecallypri NULL NULL NULL 442.84 +n stesecallypri amalgimporto #x NULL NULL 442.84 +n stesecallypri amalgimporto #x accessories NULL 442.84 +n stesecallypri amalgimporto #x accessories Men 442.84 +n stn stcallyoughtought NULL NULL NULL 443.20883534136544 +n stn stcallyoughtought corpmaxi #x NULL NULL 443.20883534136544 +n stn stcallyoughtought corpmaxi #x golf NULL 443.20883534136544 +n stn stcallyoughtought corpmaxi #x golf Sports 443.20883534136544 +ationableprieing NULL NULL NULL 443.349593495935 +ationableprieing exportiamalg #x NULL NULL 443.349593495935 +ationableprieing exportiamalg #x maternity NULL 443.349593495935 +ationableprieing exportiamalg #x maternity Women 443.349593495935 +ationoughtesepri NULL NULL NULL 443.8292682926829 +ationoughtesepri edu packunivamalg #x NULL NULL 443.8292682926829 +ationoughtesepri edu packunivamalg #x sports NULL 443.8292682926829 +ationoughtesepri edu packunivamalg #x sports Books 443.8292682926829 +oughtbarcallycallyought NULL NULL NULL 444.5889328063241 +oughtbarcallycallyought corpmaxi #x NULL NULL 444.5889328063241 +oughtbarcallycallyought corpmaxi #x golf NULL 444.5889328063241 +oughtbarcallycallyought corpmaxi #x golf Sports 444.5889328063241 +n steingcallycally NULL NULL NULL 445.0833333333333 +n steingcallycally importoscholar #x NULL NULL 445.0833333333333 +n steingcallycally importoscholar #x country NULL 445.0833333333333 +n steingcallycally importoscholar #x country Music 445.0833333333333 +ationcallyoughteing NULL NULL NULL 445.83534136546183 +ationcallyoughteing amalgedu pack #x NULL NULL 445.83534136546183 +ationcallyoughteing amalgedu pack #x womens NULL 445.83534136546183 +ationcallyoughteing amalgedu pack #x womens Shoes 445.83534136546183 +antiablebarantiought NULL NULL NULL 446.05555555555554 +antiablebarantiought scholarbrand #x NULL NULL 446.05555555555554 +antiablebarantiought scholarbrand #x custom NULL 446.05555555555554 +antiablebarantiought scholarbrand #x custom Jewelry 446.05555555555554 +ationantiationeseought NULL NULL NULL 446.33870967741933 +ationantiationeseought amalgimporto #x NULL NULL 446.33870967741933 +ationantiationeseought amalgimporto #x accessories NULL 446.33870967741933 +ationantiationeseought amalgimporto #x accessories Men 446.33870967741933 +pricallypriationought NULL NULL NULL 446.96 +pricallypriationought maxibrand #x NULL NULL 446.96 +pricallypriationought maxibrand #x mattresses NULL 446.96 +pricallypriationought maxibrand #x mattresses Home 446.96 +ationcallyn stbarought NULL NULL NULL 446.9918032786885 +ationcallyn stbarought corpbrand #x NULL NULL 446.9918032786885 +ationcallyn stbarought corpbrand #x rugs NULL 446.9918032786885 +ationcallyn stbarought corpbrand #x rugs Home 446.9918032786885 +ationeseoughtpri NULL NULL NULL 447.0769230769231 +ationeseoughtpri edu packamalg #x NULL NULL 447.0769230769231 +ationeseoughtpri edu packamalg #x swimwear NULL 447.0769230769231 +ationeseoughtpri edu packamalg #x swimwear Women 447.0769230769231 +anticallyese NULL NULL NULL 447.4313725490196 +anticallyese amalgcorp #x NULL NULL 447.4313725490196 +anticallyese amalgcorp #x birdal NULL 447.4313725490196 +anticallyese amalgcorp #x birdal Jewelry 447.4313725490196 +n stpriprioughtought NULL NULL NULL 447.6375 +n stpriprioughtought scholarbrand #x NULL NULL 447.6375 +n stpriprioughtought scholarbrand #x blinds/shades NULL 447.6375 +n stpriprioughtought scholarbrand #x blinds/shades Home 447.6375 +oughtantioughteing NULL NULL NULL 448.08097165991904 +oughtantioughteing univmaxi #x NULL NULL 448.08097165991904 +oughtantioughteing univmaxi #x pools NULL 448.08097165991904 +oughtantioughteing univmaxi #x pools Sports 448.08097165991904 +n steseeseationought NULL NULL NULL 448.1769547325103 +n steseeseationought exportiunivamalg #x NULL NULL 448.1769547325103 +n steseeseationought exportiunivamalg #x self-help NULL 448.1769547325103 +n steseeseationought exportiunivamalg #x self-help Books 448.1769547325103 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q24.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q24.sql.out new file mode 100644 index 0000000000000..e50b231441a5d --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q24.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q27a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q27a.sql.out new file mode 100644 index 0000000000000..5032ce4890dce --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q27a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 1 49.916138404648706 75.425222 191.109448 37.896850 +AAAAAAAAAAABAAAA NULL 1 67.0 37.520000 0.000000 24.010000 +AAAAAAAAAAABAAAA TN 0 67.0 37.520000 0.000000 24.010000 +AAAAAAAAAAACAAAA NULL 1 34.0 89.610000 186.400000 32.250000 +AAAAAAAAAAACAAAA TN 0 34.0 89.610000 186.400000 32.250000 +AAAAAAAAAACCAAAA NULL 1 69.0 31.085000 0.000000 7.825000 +AAAAAAAAAACCAAAA TN 0 69.0 31.085000 0.000000 7.825000 +AAAAAAAAAACDAAAA NULL 1 36.0 107.280000 0.000000 54.063333 +AAAAAAAAAACDAAAA TN 0 36.0 107.280000 0.000000 54.063333 +AAAAAAAAAADBAAAA NULL 1 21.0 74.470000 0.000000 44.680000 +AAAAAAAAAADBAAAA TN 0 21.0 74.470000 0.000000 44.680000 +AAAAAAAAAAEBAAAA NULL 1 47.0 30.540000 125.010000 20.460000 +AAAAAAAAAAEBAAAA TN 0 47.0 30.540000 125.010000 20.460000 +AAAAAAAAAAEEAAAA NULL 1 26.5 100.840000 362.865000 44.030000 +AAAAAAAAAAEEAAAA TN 0 26.5 100.840000 362.865000 44.030000 +AAAAAAAAAAFCAAAA NULL 1 50.0 37.055000 0.000000 7.555000 +AAAAAAAAAAFCAAAA TN 0 50.0 37.055000 0.000000 7.555000 +AAAAAAAAAAGBAAAA NULL 1 14.0 55.130000 0.000000 50.160000 +AAAAAAAAAAGBAAAA TN 0 14.0 55.130000 0.000000 50.160000 +AAAAAAAAAAHBAAAA NULL 1 50.0 106.065000 0.000000 29.695000 +AAAAAAAAAAHBAAAA TN 0 50.0 106.065000 0.000000 29.695000 +AAAAAAAAAAIAAAAA NULL 1 67.0 68.610000 0.000000 26.405000 +AAAAAAAAAAIAAAAA TN 0 67.0 68.610000 0.000000 26.405000 +AAAAAAAAAAIDAAAA NULL 1 37.333333333333336 78.440000 0.000000 51.863333 +AAAAAAAAAAIDAAAA TN 0 37.333333333333336 78.440000 0.000000 51.863333 +AAAAAAAAAAJBAAAA NULL 1 55.0 131.290000 0.000000 36.760000 +AAAAAAAAAAJBAAAA TN 0 55.0 131.290000 0.000000 36.760000 +AAAAAAAAAAKBAAAA NULL 1 70.0 66.120000 0.000000 21.150000 +AAAAAAAAAAKBAAAA TN 0 70.0 66.120000 0.000000 21.150000 +AAAAAAAAAALCAAAA NULL 1 93.0 104.200000 0.000000 6.250000 +AAAAAAAAAALCAAAA TN 0 93.0 104.200000 0.000000 6.250000 +AAAAAAAAAALDAAAA NULL 1 61.5 150.645000 0.000000 57.335000 +AAAAAAAAAALDAAAA TN 0 61.5 150.645000 0.000000 57.335000 +AAAAAAAAAANAAAAA NULL 1 52.0 2.510000 0.000000 2.280000 +AAAAAAAAAANAAAAA TN 0 52.0 2.510000 0.000000 2.280000 +AAAAAAAAAAOAAAAA NULL 1 23.5 17.590000 0.000000 12.130000 +AAAAAAAAAAOAAAAA TN 0 23.5 17.590000 0.000000 12.130000 +AAAAAAAAAAOCAAAA NULL 1 30.0 22.380000 0.000000 4.690000 +AAAAAAAAAAOCAAAA TN 0 30.0 22.380000 0.000000 4.690000 +AAAAAAAAAAPCAAAA NULL 1 92.0 75.630000 0.000000 64.280000 +AAAAAAAAAAPCAAAA TN 0 92.0 75.630000 0.000000 64.280000 +AAAAAAAAABAAAAAA NULL 1 10.0 45.790000 0.000000 43.950000 +AAAAAAAAABAAAAAA TN 0 10.0 45.790000 0.000000 43.950000 +AAAAAAAAABABAAAA NULL 1 63.0 21.890000 0.000000 19.700000 +AAAAAAAAABABAAAA TN 0 63.0 21.890000 0.000000 19.700000 +AAAAAAAAABBAAAAA NULL 1 26.5 78.785000 0.000000 35.615000 +AAAAAAAAABBAAAAA TN 0 26.5 78.785000 0.000000 35.615000 +AAAAAAAAABBDAAAA NULL 1 14.0 67.910000 0.000000 36.670000 +AAAAAAAAABBDAAAA TN 0 14.0 67.910000 0.000000 36.670000 +AAAAAAAAABCBAAAA NULL 1 42.5 72.900000 0.000000 44.685000 +AAAAAAAAABCBAAAA TN 0 42.5 72.900000 0.000000 44.685000 +AAAAAAAAABCCAAAA NULL 1 50.0 90.835000 0.000000 42.605000 +AAAAAAAAABCCAAAA TN 0 50.0 90.835000 0.000000 42.605000 +AAAAAAAAABDBAAAA NULL 1 40.0 10.100000 0.000000 4.240000 +AAAAAAAAABDBAAAA TN 0 40.0 10.100000 0.000000 4.240000 +AAAAAAAAABDEAAAA NULL 1 67.0 85.420000 0.000000 31.600000 +AAAAAAAAABDEAAAA TN 0 67.0 85.420000 0.000000 31.600000 +AAAAAAAAABECAAAA NULL 1 78.5 36.280000 1673.320000 25.170000 +AAAAAAAAABECAAAA TN 0 78.5 36.280000 1673.320000 25.170000 +AAAAAAAAABEDAAAA NULL 1 60.0 92.410000 2270.190000 58.210000 +AAAAAAAAABEDAAAA TN 0 60.0 92.410000 2270.190000 58.210000 +AAAAAAAAABGAAAAA NULL 1 57.0 11.390000 201.900000 3.980000 +AAAAAAAAABGAAAAA TN 0 57.0 11.390000 201.900000 3.980000 +AAAAAAAAABGDAAAA NULL 1 74.0 174.750000 232.650000 5.240000 +AAAAAAAAABGDAAAA TN 0 74.0 174.750000 232.650000 5.240000 +AAAAAAAAABHDAAAA NULL 1 34.5 120.730000 0.000000 61.460000 +AAAAAAAAABHDAAAA TN 0 34.5 120.730000 0.000000 61.460000 +AAAAAAAAABIBAAAA NULL 1 4.0 12.680000 5.040000 10.520000 +AAAAAAAAABIBAAAA TN 0 4.0 12.680000 5.040000 10.520000 +AAAAAAAAABICAAAA NULL 1 20.5 22.420000 0.000000 20.300000 +AAAAAAAAABICAAAA TN 0 20.5 22.420000 0.000000 20.300000 +AAAAAAAAABJBAAAA NULL 1 38.0 2.850000 0.000000 0.880000 +AAAAAAAAABJBAAAA TN 0 38.0 2.850000 0.000000 0.880000 +AAAAAAAAABKCAAAA NULL 1 8.0 73.980000 0.000000 51.780000 +AAAAAAAAABKCAAAA TN 0 8.0 73.980000 0.000000 51.780000 +AAAAAAAAABLCAAAA NULL 1 49.0 96.580000 0.000000 28.970000 +AAAAAAAAABLCAAAA TN 0 49.0 96.580000 0.000000 28.970000 +AAAAAAAAABMAAAAA NULL 1 50.0 136.990000 838.320000 69.860000 +AAAAAAAAABMAAAAA TN 0 50.0 136.990000 838.320000 69.860000 +AAAAAAAAABMBAAAA NULL 1 76.0 120.520000 0.000000 83.150000 +AAAAAAAAABMBAAAA TN 0 76.0 120.520000 0.000000 83.150000 +AAAAAAAAABNAAAAA NULL 1 17.0 91.190000 0.000000 16.410000 +AAAAAAAAABNAAAAA TN 0 17.0 91.190000 0.000000 16.410000 +AAAAAAAAABNCAAAA NULL 1 61.0 13.080000 16.650000 0.650000 +AAAAAAAAABNCAAAA TN 0 61.0 13.080000 16.650000 0.650000 +AAAAAAAAABPAAAAA NULL 1 30.0 118.380000 623.160000 46.160000 +AAAAAAAAABPAAAAA TN 0 30.0 118.380000 623.160000 46.160000 +AAAAAAAAABPBAAAA NULL 1 30.0 NULL NULL 17.350000 +AAAAAAAAABPBAAAA TN 0 30.0 NULL NULL 17.350000 +AAAAAAAAACAAAAAA NULL 1 20.0 3.710000 0.000000 0.250000 +AAAAAAAAACAAAAAA TN 0 20.0 3.710000 0.000000 0.250000 +AAAAAAAAACACAAAA NULL 1 66.0 117.590000 0.000000 34.100000 +AAAAAAAAACACAAAA TN 0 66.0 117.590000 0.000000 34.100000 +AAAAAAAAACADAAAA NULL 1 79.0 171.850000 0.000000 127.160000 +AAAAAAAAACADAAAA TN 0 79.0 171.850000 0.000000 127.160000 +AAAAAAAAACBBAAAA NULL 1 81.0 88.700000 0.000000 13.300000 +AAAAAAAAACBBAAAA TN 0 81.0 88.700000 0.000000 13.300000 +AAAAAAAAACBCAAAA NULL 1 26.0 62.780000 1311.600000 61.520000 +AAAAAAAAACBCAAAA TN 0 26.0 62.780000 1311.600000 61.520000 +AAAAAAAAACBEAAAA NULL 1 60.0 84.015000 0.000000 33.495000 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q34.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q34.sql.out new file mode 100644 index 0000000000000..71b7844e8e93a --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q34.sql.out @@ -0,0 +1,223 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL Y 47915 15 +NULL NULL NULL NULL 126143 15 +NULL NULL NULL NULL 215293 15 +NULL NULL Mrs. N 120593 15 +NULL Rubin Sir NULL 30056 15 +Adler Justin Sir Y 226187 15 +Allen Rose Mrs. N 179476 16 +Anderson Marvin Dr. N 211012 16 +Andrews Jacob Mr. N 67111 16 +Andrews Samuel Dr. Y 139993 16 +Angel Kevin Mr. Y 106628 15 +Ashley Linda Mrs. Y 82173 15 +Baca Dorothy Mrs. N 64890 15 +Baker Jamie Dr. Y 9916 15 +Banks Leroy Sir N 206730 15 +Barber Dianna Mrs. Y 119959 16 +Barksdale Joann Miss Y 138994 15 +Barnes Ruth Dr. N 84038 15 +Barney Samuel Sir N 15288 15 +Barnhart Charley Mr. Y 166576 15 +Barone Seth Mr. Y 162374 15 +Barrett David Sir N 189879 15 +Bartels Elmer Sir Y 114760 16 +Bear Scott Sir Y 82291 15 +Beers Kendra Dr. NULL 137960 15 +Belcher James Sir Y 239470 16 +Bell Carrie Miss N 5527 15 +Bell Matthew Dr. N 20400 15 +Benjamin Consuelo Ms. Y 201086 15 +Bergman Joann Miss N 177052 15 +Brooks Robert Sir N 155576 16 +Byrd Kelly Sir N 165115 16 +Cagle Jennifer Miss N 163129 15 +Campbell Robert Mr. N 8964 15 +Cardona Robert Mr. N 200501 15 +Carter Wendy Ms. N 96439 15 +Carver Bernard Mr. Y 194943 16 +Chen Wanita Miss N 137713 16 +Christensen Larry Dr. Y 58094 15 +Cochrane Anne Mrs. N 208347 16 +Coleman Inez Dr. Y 88249 16 +Coleman John Mr. N 49444 15 +Colon Anna Dr. Y 143694 15 +Conley Roxie Dr. N 196663 15 +Cook Adam Ms. Y 167339 15 +Cote Justin Dr. N 93466 15 +Council Donald Sir Y 102958 15 +Cramer Linda Ms. N 126628 15 +Crittenden Amie Ms. N 138787 15 +Cruz James Mr. Y 201430 15 +Cuellar Oscar Mr. Y 86781 16 +Cullen Larry Mr. Y 221242 16 +Cushing Antonia Mrs. Y 118927 15 +Davis Gordon Dr. N 227822 15 +Davis Myrtle Dr. Y 37430 15 +Decker Vera Miss Y 75737 16 +Diamond Fernando Dr. N 216391 15 +Diaz Walton Mr. N 131135 16 +Dickinson Steven Mr. N 8057 16 +Douglas Lester Sir N 26043 15 +Dove Garry Dr. N 152171 16 +Drake Rosetta Dr. Y 238040 15 +Dumas Travis Mr. Y 94154 15 +Duncan Olivia Dr. Y 102032 15 +Durham Andrea Dr. Y 144734 15 +Dutton Gay Miss Y 110886 15 +Ellis Karen Miss N 229706 16 +Ely Cesar Dr. Y 36054 16 +Etheridge Mike Dr. N 19648 15 +Farmer Eugenia Miss Y 98187 16 +Farrow Kathy Miss Y 200078 15 +Faulkner Lakeisha Dr. Y 178393 16 +Faulkner Robert Dr. N 109423 15 +Felton David Mr. N 97807 16 +Ferreira Christine Mrs. Y 155822 15 +Finn Robert Mr. N 38057 15 +Finney Crystal Miss Y 158304 15 +Fischer Tamara Mrs. N 66790 15 +Foote Roy Sir N 68086 15 +Foreman Autumn Mrs. Y 164060 15 +Funk Marvin Sir Y 61516 15 +Garcia Christopher Sir Y 181616 16 +Garcia Karen Miss N 236987 15 +Garcia Robert Dr. N 172185 16 +Garland Michael Mr. N 234421 15 +Gaylord Keith Mr. Y 123333 16 +Gifford Mark Mr. N 225973 16 +Gilbert NULL Sir N 16844 15 +Gilmore Austin Dr. Y 239871 15 +Goldsmith Bernice Ms. Y 2347 15 +Good Nancy Dr. N 132655 15 +Goodman NULL NULL N 71903 15 +Gower Nettie Miss N 10576 15 +Gray Evelyn Miss N 157486 15 +Hammond Roger Sir Y 54884 16 +Hardin Kimberly Dr. N 192424 16 +Harp Vance Mr. N 199017 15 +Harper Madeline Dr. N 173835 16 +Harris Tammy Dr. N 217761 16 +Hartmann Joey Dr. N 230915 15 +Hayes David Sir N 82274 15 +Haynes Sara Miss Y 139168 16 +Heath Matthew Dr. N 30710 15 +Hennessey Debbie Dr. Y 79256 15 +Herman Stella Ms. Y 33801 16 +Hernandez Max Mr. N 16015 15 +Hernandez Ruth Ms. Y 97000 15 +Hess Joseph Sir N 151336 15 +Hodges Lucas Dr. Y 163325 15 +Holland Jeremiah Dr. N 95938 16 +Jackson William Mr. Y 16425 16 +Jameson Miguel Dr. N 9181 16 +Jarrell Thomas Mr. Y 85787 16 +Johnson Julia Dr. N 27560 15 +Jones Theresa Ms. N 219765 16 +Kelly Mark Mr. Y 17039 16 +Khan Hank Mr. N 177803 15 +Kim Charlotte Dr. Y 7208 16 +Kunz Sarah Dr. N 74568 15 +Lake Robert Sir N 13264 15 +Landry Rudolph Sir N 117581 15 +Lane Luis Sir N 232302 16 +Langford Darlene Mrs. N 214891 15 +Larson Kevin Mr. Y 35053 15 +Larson Thomas Mr. N 114265 15 +Lee Malik Dr. N 20122 16 +Leonard Orlando Dr. Y 133168 15 +Lincoln Anthony Miss Y 1407 16 +Lindsey Linda Dr. N 62687 16 +Lopez Karen Dr. Y 136008 15 +Lunsford Kevin Dr. N 159120 16 +Lynch Sylvia Ms. Y 115438 15 +Lyon Michael Mr. N 140323 15 +Maestas Mabel Mrs. N 184265 15 +Magana Diann Mrs. Y 19139 15 +Manning Annamarie Ms. N 4984 16 +Marshall Felipe Sir N 138890 15 +Martin Paul Dr. N 26115 16 +Martinez Earl Sir N 108982 15 +Martinez Robert Sir Y 157672 16 +Masterson Barbara Mrs. N 231070 15 +Mata Deborah Miss Y 4323 15 +Mccoy Debbie Dr. N 91552 15 +Mcgill Tony Sir N 110030 15 +Mckeon Christina Dr. N 26190 15 +Mcnamara Linda Dr. Y 7957 15 +Means Michael Mr. Y 226164 16 +Medina Joseph Sir Y 110246 15 +Meyers Zachary Mr. Y 59549 15 +Montgomery John Mr. Y 103718 15 +Moody Miranda Ms. Y 171671 15 +Moore Mark Dr. N 191471 15 +Moran Celia Ms. Y 200691 15 +Morgan Cecelia Mrs. N 200742 15 +Morrell Chad Mr. N 93790 15 +Morse Robert Mr. N 68627 16 +Neel Audrey Ms. Y 193308 15 +Neff Sheri Mrs. Y 52556 15 +Nelson Katherine Mrs. N 110232 15 +New Suzanne Miss N 5120 16 +Nielsen Veronica Mrs. N 23905 15 +Oakley George Mr. Y 177890 15 +Parker Barbar Dr. N 57241 15 +Parker Jeff Sir N 213566 16 +Pemberton Jennifer Mrs. Y 49875 16 +Perry Robert Mr. Y 153147 16 +Phillips David Dr. N 148883 15 +Phillips Georgia NULL Y 26878 15 +Phillips Stanley Sir N 31989 15 +Pinkston Brenda Dr. N 126440 15 +Pryor Dorothy Mrs. N 213779 16 +Reed William Dr. N 145002 15 +Reynolds Amelia Ms. Y 68440 15 +Rice David Dr. Y 70484 16 +Robertson Daniel Mr. N 40407 16 +Rosales NULL NULL Y 156406 16 +Russo Cheryl Miss N 81123 15 +Sanchez Bruce Sir Y 124479 15 +Schmitz Kaitlyn Miss N 105162 15 +Sebastian Homer Dr. Y 64994 15 +Sexton Jerry Sir N 91446 15 +Sierra David Sir Y 61810 15 +Simmons Joseph Dr. N 54185 15 +Simpson Michael Sir Y 186613 16 +Simpson Shalanda Dr. Y 181123 15 +Singleton Andrew Ms. N 45464 15 +Smith Danny Dr. Y 143297 15 +Smith Edward Sir Y 81178 16 +Smith Hung Sir N 44710 15 +Smith Kimberly Mrs. Y 174638 15 +Smith Vern Sir N 50960 15 +Sosa Leah Ms. Y 77106 16 +Sparks Erick Dr. N 220337 15 +Taylor Kenneth Dr. Y 194337 15 +Todd Linda Ms. Y 235816 15 +Trout Harley Mr. Y 214547 15 +Urban NULL NULL NULL 214898 15 +Varner Elsie Ms. N 199602 16 +Vazquez Bill Dr. Y 62049 15 +Velazquez William Dr. N 46239 15 +Wagner Barbara Ms. Y 233595 15 +Ward Anna Miss N 52941 16 +Watkins Rosa Miss Y 152190 16 +Welch James Dr. Y 51441 16 +West Teresa Ms. N 233179 16 +White Maurice Mr. N 10107 15 +Williams John Mr. Y 84783 15 +Williams Robert Mr. Y 41233 15 +Williamson Ruth Mrs. Y 86369 15 +Wilson Joel Sir Y 91826 16 +Wilson John Sir Y 26543 15 +Wilson Mariano Mr. Y 67472 16 +Winkler Jose Dr. Y 78400 15 +Winter Cora Mrs. N 8978 16 +Wood Marcia Ms. Y 219276 16 +Wood Michelle Mrs. N 39560 15 +Wright Richie Sir Y 106818 15 +Young William Mr. Y 51127 15 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q35.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q35.sql.out new file mode 100644 index 0000000000000..0c6741654b7ee --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q35.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL F D 0 1 0.0 0 0 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F D 0 1 0.0 0 0 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F D 0 1 0.0 0 0 5 1 5.0 5 5 2 1 2.0 2 2 +NULL F D 0 1 0.0 0 0 6 1 6.0 6 6 4 1 4.0 4 4 +NULL F D 1 1 1.0 1 1 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F D 1 1 1.0 1 1 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F D 1 1 1.0 1 1 4 1 4.0 4 4 5 1 5.0 5 5 +NULL F D 2 1 2.0 2 2 0 1 0.0 0 0 4 1 4.0 4 4 +NULL F D 2 1 2.0 2 2 1 1 1.0 1 1 3 1 3.0 3 3 +NULL F D 2 1 2.0 2 2 6 1 6.0 6 6 1 1 1.0 1 1 +NULL F D 3 1 3.0 3 3 3 1 3.0 3 3 2 1 2.0 2 2 +NULL F D 3 1 3.0 3 3 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F D 3 1 3.0 3 3 4 1 4.0 4 4 1 1 1.0 1 1 +NULL F D 4 1 4.0 4 4 0 1 0.0 0 0 3 1 3.0 3 3 +NULL F D 4 1 4.0 4 4 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F D 4 1 4.0 4 4 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F D 4 1 4.0 4 4 5 1 5.0 5 5 6 1 6.0 6 6 +NULL F D 5 1 5.0 5 5 4 1 4.0 4 4 3 1 3.0 3 3 +NULL F D 5 1 5.0 5 5 5 1 5.0 5 5 2 1 2.0 2 2 +NULL F D 6 1 6.0 6 6 1 1 1.0 1 1 3 1 3.0 3 3 +NULL F D 6 1 6.0 6 6 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F D 6 1 6.0 6 6 4 1 4.0 4 4 1 1 1.0 1 1 +NULL F M 0 1 0.0 0 0 5 1 5.0 5 5 5 1 5.0 5 5 +NULL F M 1 1 1.0 1 1 3 1 3.0 3 3 0 1 0.0 0 0 +NULL F M 1 1 1.0 1 1 6 1 6.0 6 6 0 1 0.0 0 0 +NULL F M 1 1 1.0 1 1 6 1 6.0 6 6 1 1 1.0 1 1 +NULL F M 2 1 2.0 2 2 2 1 2.0 2 2 6 1 6.0 6 6 +NULL F M 2 1 2.0 2 2 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F M 3 1 3.0 3 3 2 1 2.0 2 2 1 1 1.0 1 1 +NULL F M 3 1 3.0 3 3 5 1 5.0 5 5 0 1 0.0 0 0 +NULL F M 3 1 3.0 3 3 5 1 5.0 5 5 1 1 1.0 1 1 +NULL F M 4 1 4.0 4 4 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F M 4 1 4.0 4 4 2 1 2.0 2 2 1 1 1.0 1 1 +NULL F M 4 1 4.0 4 4 3 1 3.0 3 3 3 1 3.0 3 3 +NULL F M 5 1 5.0 5 5 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F M 6 1 6.0 6 6 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F M 6 1 6.0 6 6 5 1 5.0 5 5 6 1 6.0 6 6 +NULL F S 0 1 0.0 0 0 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F S 1 1 1.0 1 1 0 1 0.0 0 0 4 1 4.0 4 4 +NULL F S 1 1 1.0 1 1 1 1 1.0 1 1 2 1 2.0 2 2 +NULL F S 1 1 1.0 1 1 2 1 2.0 2 2 6 1 6.0 6 6 +NULL F S 1 1 1.0 1 1 5 1 5.0 5 5 5 1 5.0 5 5 +NULL F S 2 1 2.0 2 2 0 1 0.0 0 0 3 1 3.0 3 3 +NULL F S 2 2 2.0 2 4 5 2 5.0 5 10 6 2 6.0 6 12 +NULL F S 3 1 3.0 3 3 0 1 0.0 0 0 4 1 4.0 4 4 +NULL F S 3 1 3.0 3 3 2 1 2.0 2 2 1 1 1.0 1 1 +NULL F S 3 1 3.0 3 3 2 1 2.0 2 2 5 1 5.0 5 5 +NULL F S 3 1 3.0 3 3 3 1 3.0 3 3 3 1 3.0 3 3 +NULL F S 4 1 4.0 4 4 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F S 4 1 4.0 4 4 2 1 2.0 2 2 4 1 4.0 4 4 +NULL F S 5 1 5.0 5 5 6 1 6.0 6 6 0 1 0.0 0 0 +NULL F U 0 1 0.0 0 0 1 1 1.0 1 1 3 1 3.0 3 3 +NULL F U 0 1 0.0 0 0 3 1 3.0 3 3 0 1 0.0 0 0 +NULL F U 1 1 1.0 1 1 3 1 3.0 3 3 2 1 2.0 2 2 +NULL F U 1 1 1.0 1 1 5 1 5.0 5 5 6 1 6.0 6 6 +NULL F U 2 1 2.0 2 2 0 1 0.0 0 0 1 1 1.0 1 1 +NULL F U 2 1 2.0 2 2 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F U 3 2 3.0 3 6 1 2 1.0 1 2 6 2 6.0 6 12 +NULL F U 4 1 4.0 4 4 0 1 0.0 0 0 4 1 4.0 4 4 +NULL F U 5 1 5.0 5 5 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F U 6 1 6.0 6 6 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F U 6 1 6.0 6 6 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F U 6 1 6.0 6 6 5 1 5.0 5 5 0 1 0.0 0 0 +NULL F U 6 1 6.0 6 6 5 1 5.0 5 5 6 1 6.0 6 6 +NULL F W 0 1 0.0 0 0 0 1 0.0 0 0 4 1 4.0 4 4 +NULL F W 0 1 0.0 0 0 5 1 5.0 5 5 5 1 5.0 5 5 +NULL F W 1 1 1.0 1 1 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F W 2 1 2.0 2 2 0 1 0.0 0 0 5 1 5.0 5 5 +NULL F W 3 1 3.0 3 3 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F W 3 1 3.0 3 3 6 1 6.0 6 6 6 1 6.0 6 6 +NULL F W 4 1 4.0 4 4 3 1 3.0 3 3 1 1 1.0 1 1 +NULL F W 5 1 5.0 5 5 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F W 5 1 5.0 5 5 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F W 5 1 5.0 5 5 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F W 5 1 5.0 5 5 4 1 4.0 4 4 6 1 6.0 6 6 +NULL F W 6 1 6.0 6 6 0 1 0.0 0 0 5 1 5.0 5 5 +NULL F W 6 1 6.0 6 6 2 1 2.0 2 2 3 1 3.0 3 3 +NULL F W 6 1 6.0 6 6 5 1 5.0 5 5 5 1 5.0 5 5 +NULL M D 0 1 0.0 0 0 3 1 3.0 3 3 0 1 0.0 0 0 +NULL M D 1 1 1.0 1 1 3 1 3.0 3 3 0 1 0.0 0 0 +NULL M D 1 1 1.0 1 1 3 1 3.0 3 3 2 1 2.0 2 2 +NULL M D 2 1 2.0 2 2 0 1 0.0 0 0 6 1 6.0 6 6 +NULL M D 2 1 2.0 2 2 4 1 4.0 4 4 4 1 4.0 4 4 +NULL M D 2 1 2.0 2 2 5 1 5.0 5 5 3 1 3.0 3 3 +NULL M D 3 1 3.0 3 3 1 1 1.0 1 1 5 1 5.0 5 5 +NULL M D 3 1 3.0 3 3 2 1 2.0 2 2 3 1 3.0 3 3 +NULL M D 4 1 4.0 4 4 5 1 5.0 5 5 2 1 2.0 2 2 +NULL M D 6 1 6.0 6 6 1 1 1.0 1 1 6 1 6.0 6 6 +NULL M D 6 1 6.0 6 6 3 1 3.0 3 3 1 1 1.0 1 1 +NULL M M 0 1 0.0 0 0 0 1 0.0 0 0 1 1 1.0 1 1 +NULL M M 0 2 0.0 0 0 1 2 1.0 1 2 2 2 2.0 2 4 +NULL M M 0 1 0.0 0 0 2 1 2.0 2 2 1 1 1.0 1 1 +NULL M M 0 1 0.0 0 0 3 1 3.0 3 3 5 1 5.0 5 5 +NULL M M 0 1 0.0 0 0 5 1 5.0 5 5 0 1 0.0 0 0 +NULL M M 1 1 1.0 1 1 0 1 0.0 0 0 1 1 1.0 1 1 +NULL M M 1 1 1.0 1 1 0 1 0.0 0 0 2 1 2.0 2 2 +NULL M M 2 1 2.0 2 2 6 1 6.0 6 6 5 1 5.0 5 5 +NULL M M 3 1 3.0 3 3 5 1 5.0 5 5 1 1 1.0 1 1 +NULL M M 3 1 3.0 3 3 6 1 6.0 6 6 4 1 4.0 4 4 +NULL M M 4 1 4.0 4 4 1 1 1.0 1 1 3 1 3.0 3 3 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q35a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q35a.sql.out new file mode 100644 index 0000000000000..5d57fd93ef7d2 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q35a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL F D 0 1 0.0 0 0 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F D 1 1 1.0 1 1 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F D 1 1 1.0 1 1 4 1 4.0 4 4 3 1 3.0 3 3 +NULL F D 1 1 1.0 1 1 4 1 4.0 4 4 6 1 6.0 6 6 +NULL F D 2 1 2.0 2 2 2 1 2.0 2 2 5 1 5.0 5 5 +NULL F D 2 1 2.0 2 2 6 1 6.0 6 6 3 1 3.0 3 3 +NULL F D 3 1 3.0 3 3 0 1 0.0 0 0 2 1 2.0 2 2 +NULL F D 3 1 3.0 3 3 6 1 6.0 6 6 3 1 3.0 3 3 +NULL F D 4 1 4.0 4 4 1 1 1.0 1 1 6 1 6.0 6 6 +NULL F D 4 1 4.0 4 4 3 1 3.0 3 3 1 1 1.0 1 1 +NULL F D 4 1 4.0 4 4 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F D 4 2 4.0 4 8 5 2 5.0 5 10 4 2 4.0 4 8 +NULL F D 5 1 5.0 5 5 5 1 5.0 5 5 2 1 2.0 2 2 +NULL F D 5 1 5.0 5 5 6 1 6.0 6 6 4 1 4.0 4 4 +NULL F D 6 1 6.0 6 6 2 1 2.0 2 2 1 1 1.0 1 1 +NULL F D 6 1 6.0 6 6 3 1 3.0 3 3 0 1 0.0 0 0 +NULL F D 6 1 6.0 6 6 5 1 5.0 5 5 1 1 1.0 1 1 +NULL F D 6 1 6.0 6 6 5 1 5.0 5 5 4 1 4.0 4 4 +NULL F M 0 1 0.0 0 0 1 1 1.0 1 1 1 1 1.0 1 1 +NULL F M 0 1 0.0 0 0 2 1 2.0 2 2 6 1 6.0 6 6 +NULL F M 0 1 0.0 0 0 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F M 0 1 0.0 0 0 5 1 5.0 5 5 3 1 3.0 3 3 +NULL F M 1 1 1.0 1 1 6 1 6.0 6 6 0 1 0.0 0 0 +NULL F M 1 1 1.0 1 1 6 1 6.0 6 6 3 1 3.0 3 3 +NULL F M 3 1 3.0 3 3 0 1 0.0 0 0 1 1 1.0 1 1 +NULL F M 3 1 3.0 3 3 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F M 3 2 3.0 3 6 4 2 4.0 4 8 5 2 5.0 5 10 +NULL F M 3 1 3.0 3 3 5 1 5.0 5 5 6 1 6.0 6 6 +NULL F M 3 1 3.0 3 3 6 1 6.0 6 6 1 1 1.0 1 1 +NULL F M 4 1 4.0 4 4 3 1 3.0 3 3 2 1 2.0 2 2 +NULL F M 4 1 4.0 4 4 6 1 6.0 6 6 1 1 1.0 1 1 +NULL F M 4 1 4.0 4 4 6 1 6.0 6 6 4 1 4.0 4 4 +NULL F M 5 1 5.0 5 5 1 1 1.0 1 1 5 1 5.0 5 5 +NULL F M 5 1 5.0 5 5 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F M 6 1 6.0 6 6 0 1 0.0 0 0 5 1 5.0 5 5 +NULL F M 6 1 6.0 6 6 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F M 6 1 6.0 6 6 6 1 6.0 6 6 3 1 3.0 3 3 +NULL F S 0 1 0.0 0 0 3 1 3.0 3 3 3 1 3.0 3 3 +NULL F S 0 1 0.0 0 0 4 1 4.0 4 4 2 1 2.0 2 2 +NULL F S 0 1 0.0 0 0 5 1 5.0 5 5 1 1 1.0 1 1 +NULL F S 0 1 0.0 0 0 5 1 5.0 5 5 2 1 2.0 2 2 +NULL F S 1 1 1.0 1 1 1 1 1.0 1 1 4 1 4.0 4 4 +NULL F S 1 1 1.0 1 1 2 1 2.0 2 2 3 1 3.0 3 3 +NULL F S 1 1 1.0 1 1 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F S 2 1 2.0 2 2 0 1 0.0 0 0 6 1 6.0 6 6 +NULL F S 2 1 2.0 2 2 5 1 5.0 5 5 4 1 4.0 4 4 +NULL F S 2 1 2.0 2 2 6 1 6.0 6 6 6 1 6.0 6 6 +NULL F S 4 1 4.0 4 4 3 1 3.0 3 3 5 1 5.0 5 5 +NULL F U 0 1 0.0 0 0 2 1 2.0 2 2 0 1 0.0 0 0 +NULL F U 0 1 0.0 0 0 3 1 3.0 3 3 0 1 0.0 0 0 +NULL F U 0 1 0.0 0 0 3 1 3.0 3 3 6 1 6.0 6 6 +NULL F U 0 1 0.0 0 0 6 1 6.0 6 6 6 1 6.0 6 6 +NULL F U 1 1 1.0 1 1 0 1 0.0 0 0 6 1 6.0 6 6 +NULL F U 1 1 1.0 1 1 3 1 3.0 3 3 5 1 5.0 5 5 +NULL F U 1 1 1.0 1 1 6 1 6.0 6 6 5 1 5.0 5 5 +NULL F U 2 1 2.0 2 2 0 1 0.0 0 0 5 1 5.0 5 5 +NULL F U 2 2 2.0 2 4 1 2 1.0 1 2 0 2 0.0 0 0 +NULL F U 2 1 2.0 2 2 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F U 2 1 2.0 2 2 4 1 4.0 4 4 6 1 6.0 6 6 +NULL F U 3 1 3.0 3 3 0 1 0.0 0 0 6 1 6.0 6 6 +NULL F U 3 1 3.0 3 3 2 1 2.0 2 2 0 1 0.0 0 0 +NULL F U 4 1 4.0 4 4 0 1 0.0 0 0 6 1 6.0 6 6 +NULL F U 4 1 4.0 4 4 1 1 1.0 1 1 0 1 0.0 0 0 +NULL F U 5 1 5.0 5 5 0 1 0.0 0 0 2 1 2.0 2 2 +NULL F U 5 1 5.0 5 5 1 1 1.0 1 1 2 1 2.0 2 2 +NULL F U 5 1 5.0 5 5 4 1 4.0 4 4 4 1 4.0 4 4 +NULL F U 6 1 6.0 6 6 0 1 0.0 0 0 0 1 0.0 0 0 +NULL F U 6 1 6.0 6 6 6 1 6.0 6 6 4 1 4.0 4 4 +NULL F W 1 1 1.0 1 1 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F W 3 1 3.0 3 3 4 1 4.0 4 4 1 1 1.0 1 1 +NULL F W 4 1 4.0 4 4 3 1 3.0 3 3 4 1 4.0 4 4 +NULL F W 4 1 4.0 4 4 5 1 5.0 5 5 3 1 3.0 3 3 +NULL F W 5 1 5.0 5 5 2 1 2.0 2 2 3 1 3.0 3 3 +NULL F W 6 1 6.0 6 6 2 1 2.0 2 2 0 1 0.0 0 0 +NULL F W 6 1 6.0 6 6 2 1 2.0 2 2 2 1 2.0 2 2 +NULL F W 6 1 6.0 6 6 2 1 2.0 2 2 3 1 3.0 3 3 +NULL F W 6 1 6.0 6 6 2 1 2.0 2 2 6 1 6.0 6 6 +NULL F W 6 1 6.0 6 6 4 1 4.0 4 4 6 1 6.0 6 6 +NULL F W 6 1 6.0 6 6 5 1 5.0 5 5 0 1 0.0 0 0 +NULL F W 6 1 6.0 6 6 6 1 6.0 6 6 5 1 5.0 5 5 +NULL M D 0 1 0.0 0 0 0 1 0.0 0 0 6 1 6.0 6 6 +NULL M D 0 1 0.0 0 0 3 1 3.0 3 3 0 1 0.0 0 0 +NULL M D 0 1 0.0 0 0 5 1 5.0 5 5 5 1 5.0 5 5 +NULL M D 1 1 1.0 1 1 2 1 2.0 2 2 0 1 0.0 0 0 +NULL M D 1 1 1.0 1 1 2 1 2.0 2 2 1 1 1.0 1 1 +NULL M D 1 1 1.0 1 1 2 1 2.0 2 2 6 1 6.0 6 6 +NULL M D 1 1 1.0 1 1 3 1 3.0 3 3 4 1 4.0 4 4 +NULL M D 2 1 2.0 2 2 1 1 1.0 1 1 0 1 0.0 0 0 +NULL M D 2 1 2.0 2 2 1 1 1.0 1 1 4 1 4.0 4 4 +NULL M D 2 1 2.0 2 2 4 1 4.0 4 4 6 1 6.0 6 6 +NULL M D 2 1 2.0 2 2 6 1 6.0 6 6 1 1 1.0 1 1 +NULL M D 3 1 3.0 3 3 0 1 0.0 0 0 6 1 6.0 6 6 +NULL M D 3 1 3.0 3 3 3 1 3.0 3 3 0 1 0.0 0 0 +NULL M D 3 1 3.0 3 3 3 1 3.0 3 3 3 1 3.0 3 3 +NULL M D 3 1 3.0 3 3 5 1 5.0 5 5 2 1 2.0 2 2 +NULL M D 3 1 3.0 3 3 5 1 5.0 5 5 5 1 5.0 5 5 +NULL M D 4 1 4.0 4 4 4 1 4.0 4 4 5 1 5.0 5 5 +NULL M D 4 1 4.0 4 4 6 1 6.0 6 6 5 1 5.0 5 5 +NULL M D 5 1 5.0 5 5 2 1 2.0 2 2 3 1 3.0 3 3 +NULL M D 6 1 6.0 6 6 3 1 3.0 3 3 1 1 1.0 1 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q36a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q36a.sql.out new file mode 100644 index 0000000000000..a74e6211b49af --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q36a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +-0.43310777865000000000 NULL NULL 2 1 +-0.44057752675000000000 Home NULL 1 1 +-0.43759152110000000000 Music NULL 1 2 +-0.43708103961000000000 NULL NULL 1 3 +-0.43616253139000000000 Shoes NULL 1 4 +-0.43567118609000000000 Children NULL 1 5 +-0.43423932352000000000 Sports NULL 1 6 +-0.43342977300000000000 Electronics NULL 1 7 +-0.43243283121000000000 Women NULL 1 8 +-0.43164166900000000000 Men NULL 1 9 +-0.42516187690000000000 Books NULL 1 10 +-0.42448713381000000000 Jewelry NULL 1 11 +-0.73902664238792748962 NULL shirts 0 1 +-0.61125804873635587486 NULL country 0 2 +-0.53129803597069255822 NULL dresses 0 3 +-0.51266635289382758517 NULL athletic 0 4 +-0.45290387783638603924 NULL mens 0 5 +-0.41288056661656330013 NULL accessories 0 6 +-0.40784754677005682440 NULL NULL 0 7 +-0.34254844860867375832 NULL baseball 0 8 +-0.32511461675631534897 NULL infants 0 9 +-0.44733955704648003493 Books computers 0 1 +-0.44221358112622373783 Books home repair 0 2 +-0.44131129175272951442 Books romance 0 3 +-0.43954111564375046074 Books history 0 4 +-0.43921337505389731821 Books mystery 0 5 +-0.43904020269360481109 Books sports 0 6 +-0.42821476999837619396 Books travel 0 7 +-0.42609067296303848297 Books cooking 0 8 +-0.42538995145338568328 Books fiction 0 9 +-0.42446563616188232944 Books arts 0 10 +-0.42424821311884350413 Books parenting 0 11 +-0.41822014479424203008 Books reference 0 12 +-0.41350839325516811781 Books business 0 13 +-0.40935208137315013129 Books science 0 14 +-0.40159380735731858928 Books self-help 0 15 +-0.36957884843305744526 Books entertainments 0 16 +-0.44602461556731552282 Children school-uniforms 0 1 +-0.44141106040000560852 Children toddlers 0 2 +-0.43479886701046623711 Children infants 0 3 +-0.41900662971936329442 Children newborn 0 4 +-0.41526603781609697786 Children NULL 0 5 +-0.45347482218635333366 Electronics personal 0 1 +-0.44349670349829474271 Electronics stereo 0 2 +-0.44262427232850112058 Electronics automotive 0 3 +-0.44115886172705231970 Electronics portable 0 4 +-0.43972786651639318010 Electronics memory 0 5 +-0.43889275271590953040 Electronics scanners 0 6 +-0.43879181695132886061 Electronics karoke 0 7 +-0.43743655149948399284 Electronics dvd/vcr players 0 8 +-0.43737666390514154910 Electronics cameras 0 9 +-0.43390499017233926812 Electronics wireless 0 10 +-0.43163869754114299547 Electronics audio 0 11 +-0.42958938669780912634 Electronics camcorders 0 12 +-0.42872845803629855724 Electronics musical 0 13 +-0.42228240153396399656 Electronics televisions 0 14 +-0.41893847772039275795 Electronics monitors 0 15 +-0.39793878022746331540 Electronics disk drives 0 16 +-0.49051156860507320113 Home NULL 0 1 +-0.48431476750686752965 Home blinds/shades 0 2 +-0.47545837941951440918 Home bathroom 0 3 +-0.45726228921216284093 Home rugs 0 4 +-0.45540507568891021759 Home furniture 0 5 +-0.45303572267019508501 Home flatware 0 6 +-0.44755542058111800358 Home tables 0 7 +-0.44419847780930149402 Home wallpaper 0 8 +-0.44092345226680695671 Home glassware 0 9 +-0.43877591834074789745 Home decor 0 10 +-0.43765482553654514822 Home accent 0 11 +-0.43188199218974854630 Home bedding 0 12 +-0.43107417904272222899 Home kids 0 13 +-0.42474436355625900935 Home lighting 0 14 +-0.41783311109052416746 Home curtains/drapes 0 15 +-0.41767111806961188479 Home mattresses 0 16 +-0.40562188698541221499 Home paint 0 17 +-0.45165056505480816921 Jewelry jewelry boxes 0 1 +-0.44372227804836590137 Jewelry estate 0 2 +-0.44251815032563188894 Jewelry gold 0 3 +-0.43978127753996883542 Jewelry consignment 0 4 +-0.43821750044359339153 Jewelry custom 0 5 +-0.43439645036479672989 Jewelry bracelets 0 6 +-0.43208398325687772942 Jewelry loose stones 0 7 +-0.43060897375114375156 Jewelry diamonds 0 8 +-0.42847505748860847066 Jewelry costume 0 9 +-0.42667449062277843561 Jewelry rings 0 10 +-0.41987969011585456826 Jewelry mens watch 0 11 +-0.41624621972944533035 Jewelry semi-precious 0 12 +-0.41148949162100715771 Jewelry womens watch 0 13 +-0.39725668174847694299 Jewelry birdal 0 14 +-0.39665274051903254057 Jewelry pendants 0 15 +-0.38423525233438861010 Jewelry earings 0 16 +-0.44464388887858793403 Men shirts 0 1 +-0.43719860800637369827 Men accessories 0 2 +-0.43164606665359630905 Men sports-apparel 0 3 +-0.41530906677293519754 Men pants 0 4 +-0.38332708894803499123 Men NULL 0 5 +-0.47339698705534020269 Music NULL 0 1 +-0.44193214675249008923 Music rock 0 2 +-0.44008174913565459246 Music country 0 3 +-0.43863444992223641373 Music pop 0 4 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q47.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q47.sql.out new file mode 100644 index 0000000000000..0bdc42a4751cb --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q47.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Shoes 1999 3 5607.487500 2197.48 3271.66 2831.67 +Shoes 1999 2 5643.938333 2393.31 4463.11 2652.44 +Shoes 1999 4 5640.362500 2416.57 3348.90 2987.78 +Men 1999 6 4702.116667 1534.95 2666.37 2514.13 +Men 1999 7 5330.618333 2218.63 3182.74 7436.83 +Shoes 1999 4 5338.852500 2233.60 3470.43 2832.41 +Music 1999 5 5139.465000 2034.96 3149.72 3648.17 +Men 1999 7 5748.707500 2645.88 3432.67 7646.91 +Men 1999 3 4915.190000 1815.92 2884.81 2956.00 +Women 1999 6 4586.300000 1508.07 2992.12 3059.37 +Shoes 1999 4 5374.032500 2322.80 2484.06 3313.69 +Men 1999 6 4596.057500 1577.69 2457.43 2439.68 +Men 1999 3 5839.670833 2825.07 3157.46 3531.36 +Men 1999 4 5342.149167 2347.82 2787.31 3588.16 +Shoes 1999 3 5643.938333 2652.44 2393.31 3008.88 +Men 1999 3 5475.719167 2515.22 2709.11 2702.85 +Shoes 1999 6 5338.852500 2388.17 2832.41 4216.10 +Shoes 1999 2 5065.706667 2119.85 3439.05 2640.59 +Shoes 1999 2 4713.009167 1774.19 3366.21 1940.05 +Shoes 1999 3 5374.032500 2484.06 2994.92 2322.80 +Men 1999 7 5342.149167 2454.72 2766.42 7665.06 +Men 1999 4 4741.993333 1879.47 3419.59 2634.29 +Music 1999 4 4723.742500 1866.30 2384.82 2931.55 +Shoes 1999 4 4732.205000 1875.08 2686.46 3422.03 +Children 1999 6 4849.967500 2002.59 2590.23 3380.67 +Men 1999 6 4920.419167 2077.36 3402.55 3347.44 +Men 1999 4 4596.057500 1762.17 2728.14 2457.43 +Children 1999 4 4739.001667 1923.04 2309.91 2849.64 +Music 1999 3 4816.848333 2010.47 2539.57 2940.38 +Shoes 1999 2 4707.697500 1903.70 2693.49 3474.34 +Shoes 1999 2 4443.995833 1642.32 3972.50 2319.04 +Men 1999 6 5839.670833 3053.78 3151.27 3622.65 +Shoes 1999 4 5607.487500 2831.67 2197.48 4187.54 +Men 1999 3 5598.894167 2824.13 3154.80 3135.60 +Shoes 1999 3 4713.009167 1940.05 1774.19 2496.18 +Men 1999 4 5475.719167 2702.85 2515.22 4364.56 +Shoes 1999 4 4596.537500 1825.00 2777.56 3234.34 +Music 1999 6 4332.550833 1563.26 2484.37 2460.11 +Men 1999 2 5102.436667 2333.32 3417.85 2536.68 +Men 1999 2 5475.719167 2709.11 4740.72 2515.22 +Men 1999 6 5475.719167 2723.08 4364.56 4000.42 +Shoes 1999 2 5338.852500 2587.72 3837.18 3470.43 +Children 1999 6 4426.704167 1683.86 2186.50 1970.76 +Shoes 1999 6 5643.938333 2902.72 3046.01 3994.18 +Shoes 1999 6 5374.032500 2637.31 3313.69 3158.65 +Men 1999 3 4920.419167 2183.89 3598.63 2841.78 +Men 1999 3 5330.618333 2602.65 2613.45 2624.00 +Women 1999 6 4366.627500 1640.10 3159.43 2882.91 +Music 1999 2 5139.465000 2413.63 4627.89 2773.28 +Men 1999 2 4929.611667 2207.89 3940.06 2246.76 +Women 1999 2 4551.444167 1833.08 3729.30 2940.96 +Men 1999 2 5330.618333 2613.45 3859.71 2602.65 +Children 1999 2 4640.935833 1927.95 3524.16 2584.03 +Children 1999 7 4565.828333 1853.57 2846.40 6447.36 +Women 1999 6 4551.444167 1840.98 2478.75 3174.74 +Men 1999 4 5330.618333 2624.00 2602.65 3722.18 +Children 1999 3 5024.325000 2319.29 2691.31 2350.53 +Children 1999 2 4836.831667 2132.80 3984.13 2182.12 +Children 1999 4 4640.935833 1950.03 2584.03 2919.51 +Shoes 1999 3 4791.740833 2100.87 2211.40 3278.75 +Men 1999 5 5839.670833 3151.27 3531.36 3053.78 +Men 1999 3 4929.611667 2246.76 2207.89 3208.14 +Men 1999 2 5839.670833 3157.46 3425.68 2825.07 +Men 1999 5 5311.965833 2631.11 2819.12 3022.54 +Music 1999 2 4388.770833 1714.25 2228.69 2407.93 +Shoes 1999 2 4989.784167 2315.46 4430.70 2976.26 +Children 1999 4 5024.325000 2350.53 2319.29 2984.92 +Women 1999 3 4586.300000 1917.37 3282.12 2557.46 +Children 1999 3 4836.831667 2182.12 2132.80 3466.25 +Shoes 1999 5 5640.362500 2987.78 2416.57 3437.87 +Men 1999 4 5071.261667 2425.64 2696.60 2641.17 +Shoes 1999 6 4443.995833 1801.87 2297.76 3078.40 +Children 1999 2 4480.353333 1838.88 3250.53 2375.04 +Men 1999 7 5102.436667 2464.06 2798.80 6978.09 +Shoes 1999 4 5643.938333 3008.88 2652.44 3046.01 +Men 1999 2 5071.261667 2438.50 4282.24 2696.60 +Children 1999 6 4525.318333 1896.34 2883.95 2727.36 +Children 1999 4 4212.889167 1588.72 2296.49 3077.12 +Shoes 1999 4 4884.260000 2261.01 2502.36 3210.47 +Men 1999 3 5748.707500 3132.90 3747.34 3768.23 +Shoes 1999 2 5640.362500 3027.85 4285.15 3348.90 +Music 1999 3 4490.221667 1883.06 2321.31 2211.82 +Children 1999 2 4423.506667 1816.43 3192.19 3334.11 +Music 1999 6 4894.035000 2288.21 3156.77 2845.45 +Music 1999 5 4829.768333 2225.46 3380.94 2782.42 +Women 1999 3 4268.562500 1665.68 1758.65 2082.99 +Children 1999 3 4267.374167 1667.03 2579.77 2802.05 +Men 1999 3 4987.821667 2387.78 2962.40 2928.83 +Women 1999 3 4309.575833 1710.42 2004.22 2742.41 +Shoes 1999 5 5643.938333 3046.01 3008.88 2902.72 +Music 1999 4 4518.543333 1922.23 2683.26 2227.02 +Children 1999 3 4441.555000 1846.53 3827.47 3623.30 +Men 1999 5 5748.707500 3156.57 3768.23 3432.67 +Music 1999 2 4302.831667 1712.70 3561.46 2414.41 +Women 1999 7 4494.382500 1904.98 2308.58 5603.14 +Women 1999 4 4582.713333 1994.71 2408.40 2321.48 +Music 1999 3 4426.623333 1840.51 2707.80 3147.10 +Shoes 1999 2 4791.740833 2211.40 3912.90 2100.87 +Shoes 1999 7 5640.362500 3062.31 3437.87 6376.30 +Children 1999 3 4733.152500 2155.79 2710.50 2685.74 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q49.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q49.sql.out new file mode 100644 index 0000000000000..31b31b7bdc02b --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q49.sql.out @@ -0,0 +1,37 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +catalog 16735 0.50505050505050505051 1 1 +catalog 12633 0.69662921348314606742 2 2 +catalog 13967 0.70000000000000000000 3 3 +catalog 12819 0.70129870129870129870 4 8 +catalog 16155 0.72043010752688172043 5 4 +catalog 17681 0.75268817204301075269 6 5 +catalog 5975 0.76404494382022471910 7 6 +catalog 11451 0.76744186046511627907 8 7 +catalog 1689 0.80219780219780219780 9 9 +catalog 10311 0.81818181818181818182 10 10 +store 5111 0.78947368421052631579 1 1 +store 11073 0.83505154639175257732 2 3 +store 14429 0.84782608695652173913 3 2 +store 15927 0.86419753086419753086 4 4 +store 10171 0.86868686868686868687 5 5 +store 12783 0.88775510204081632653 6 6 +store 11075 0.89743589743589743590 7 7 +store 12889 0.95652173913043478261 8 8 +store 1939 0.99000000000000000000 9 9 +store 4333 1.00000000000000000000 10 10 +store 10455 1.00000000000000000000 10 10 +store 12975 1.00000000000000000000 10 10 +web 10485 0.48863636363636363636 1 1 +web 4483 0.52688172043010752688 2 2 +web 8833 0.58241758241758241758 3 3 +web 1165 0.61458333333333333333 4 4 +web 17197 0.73076923076923076923 5 5 +web 10319 0.73469387755102040816 6 6 +web 13159 0.75257731958762886598 7 7 +web 9629 0.77894736842105263158 8 8 +web 5909 0.78378378378378378378 9 9 +web 7057 0.86746987951807228916 10 10 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q51a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q51a.sql.out new file mode 100644 index 0000000000000..62b22b5ca706f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q51a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +9 2001-01-20 155.31 NULL 155.31 4.26 +9 2001-02-01 168.10 28.81 168.10 28.81 +9 2001-02-04 NULL 34.51 168.10 34.51 +9 2001-02-15 NULL 144.79 168.10 144.79 +9 2001-04-02 249.70 NULL 249.70 181.21 +9 2001-04-09 NULL 183.64 249.70 183.64 +9 2001-04-17 260.21 230.47 260.21 230.47 +29 2001-01-07 145.14 NULL 145.14 12.18 +29 2001-01-10 NULL 53.52 145.14 53.52 +29 2001-01-11 NULL 55.81 145.14 55.81 +29 2001-02-05 NULL 88.85 145.14 88.85 +29 2001-02-08 NULL 140.80 145.14 140.80 +29 2001-02-28 215.28 NULL 215.28 212.84 +29 2001-03-30 340.91 NULL 340.91 279.82 +29 2001-04-07 NULL 306.91 340.91 306.91 +29 2001-05-26 392.06 389.83 392.06 389.83 +29 2001-06-12 NULL 391.96 392.06 391.96 +29 2001-06-29 580.42 NULL 580.42 407.31 +29 2001-07-08 NULL 429.12 580.42 429.12 +29 2001-07-12 NULL 444.37 580.42 444.37 +29 2001-07-26 NULL 501.13 580.42 501.13 +29 2001-07-28 NULL 523.55 580.42 523.55 +29 2001-07-31 NULL 547.65 580.42 547.65 +29 2001-09-15 755.81 NULL 755.81 710.20 +29 2001-09-16 NULL 742.83 755.81 742.83 +31 2001-01-23 160.83 143.12 160.83 143.12 +31 2001-01-24 247.53 NULL 247.53 143.12 +31 2001-02-04 NULL 143.12 247.53 143.12 +31 2001-02-07 NULL 166.46 247.53 166.46 +31 2001-02-15 NULL 224.58 247.53 224.58 +31 2001-02-22 NULL 243.99 247.53 243.99 +31 2001-02-26 NULL 245.24 247.53 245.24 +33 2001-02-06 143.86 NULL 143.86 100.10 +33 2001-03-06 260.39 NULL 260.39 100.10 +33 2001-03-08 311.65 NULL 311.65 100.10 +33 2001-03-17 NULL 166.59 311.65 166.59 +33 2001-04-04 NULL 195.94 311.65 195.94 +33 2001-04-11 NULL 218.41 311.65 218.41 +33 2001-04-15 NULL 258.16 311.65 258.16 +33 2001-04-26 NULL 260.73 311.65 260.73 +35 2001-04-11 218.63 NULL 218.63 175.11 +35 2001-04-13 NULL 200.14 218.63 200.14 +35 2001-04-15 NULL 213.01 218.63 213.01 +35 2001-04-21 236.74 NULL 236.74 213.01 +35 2001-05-12 250.14 NULL 250.14 213.01 +35 2001-06-03 315.73 NULL 315.73 213.01 +35 2001-06-09 NULL 213.01 315.73 213.01 +35 2001-06-12 350.47 NULL 350.47 213.01 +35 2001-06-16 NULL 240.39 350.47 240.39 +35 2001-06-29 NULL 251.30 350.47 251.30 +35 2001-07-03 NULL 251.62 350.47 251.62 +35 2001-07-06 NULL 279.62 350.47 279.62 +35 2001-07-10 NULL 281.30 350.47 281.30 +35 2001-07-19 NULL 285.09 350.47 285.09 +35 2001-07-22 NULL 306.31 350.47 306.31 +35 2001-07-28 421.29 NULL 421.29 306.31 +35 2001-07-29 434.60 NULL 434.60 306.31 +37 2001-03-07 104.29 NULL 104.29 100.87 +47 2001-01-01 7.00 5.55 7.00 5.55 +47 2001-01-15 72.85 NULL 72.85 48.96 +47 2001-01-22 NULL 58.39 72.85 58.39 +47 2001-02-17 139.16 NULL 139.16 119.31 +49 2001-05-05 361.90 NULL 361.90 340.45 +49 2001-05-21 397.02 NULL 397.02 340.45 +49 2001-05-25 479.86 NULL 479.86 340.45 +49 2001-06-03 NULL 350.29 479.86 350.29 +49 2001-06-10 488.44 NULL 488.44 350.29 +49 2001-06-20 NULL 362.21 488.44 362.21 +49 2001-06-28 527.37 NULL 527.37 362.21 +49 2001-07-19 NULL 490.09 527.37 490.09 +49 2001-07-23 532.83 NULL 532.83 490.09 +49 2001-07-26 NULL 511.76 532.83 511.76 +49 2001-07-31 556.70 NULL 556.70 511.76 +51 2001-03-24 142.03 NULL 142.03 141.46 +51 2001-04-11 221.62 NULL 221.62 208.13 +53 2001-02-01 150.99 NULL 150.99 15.99 +53 2001-02-03 NULL 92.68 150.99 92.68 +53 2001-02-09 NULL 96.53 150.99 96.53 +53 2001-02-18 NULL 98.68 150.99 98.68 +53 2001-02-20 NULL 129.22 150.99 129.22 +57 2001-03-15 147.84 NULL 147.84 108.81 +65 2001-01-07 NULL 7.39 143.43 7.39 +65 2001-01-08 NULL 21.35 143.43 21.35 +65 2001-01-16 NULL 102.46 143.43 102.46 +67 2001-02-02 120.54 13.19 120.54 13.19 +67 2001-02-19 219.36 NULL 219.36 120.94 +67 2001-03-12 NULL 203.25 219.36 203.25 +67 2001-04-13 330.07 NULL 330.07 277.79 +67 2001-04-23 NULL 284.23 330.07 284.23 +67 2001-04-27 NULL 290.21 330.07 290.21 +67 2001-04-28 NULL 320.26 330.07 320.26 +69 2001-02-22 54.17 NULL 54.17 36.00 +69 2001-02-27 NULL 45.08 54.17 45.08 +73 2001-01-18 184.35 NULL 184.35 178.75 +73 2001-01-19 185.11 NULL 185.11 178.75 +73 2001-02-09 NULL 180.42 185.11 180.42 +75 2001-01-15 NULL 9.11 19.68 9.11 +75 2001-01-31 36.37 NULL 36.37 9.11 +75 2001-02-03 NULL 14.06 36.37 14.06 +83 2001-02-03 72.95 NULL 72.95 55.10 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q57.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q57.sql.out new file mode 100644 index 0000000000000..ea23526b956ee --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q57.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Music amalgscholar #x 1999 2 6662.669167 1961.57 4348.07 3386.25 +Shoes amalgedu pack #x 1999 2 6493.071667 2044.05 4348.88 3443.20 +Shoes exportiedu pack #x 1999 3 7416.141667 2980.15 4654.22 5157.83 +Children importoexporti #x 1999 4 6577.143333 2152.15 3291.07 3659.32 +Shoes importoedu pack #x 1999 6 6926.960833 2523.33 4014.93 4254.99 +Men importoimporto #x 1999 2 6707.315833 2449.22 4311.11 3583.31 +Men amalgimporto #x 1999 4 7098.680833 2965.42 3526.45 4923.53 +Men exportiimporto #x 1999 2 7146.240000 3013.99 6183.83 5091.17 +Children amalgexporti #x 1999 4 6364.467500 2270.79 3330.83 3817.50 +Men edu packimporto #x 1999 4 7386.333333 3329.74 3488.01 4860.20 +Men edu packimporto #x 1999 2 7386.333333 3347.65 4007.40 3488.01 +Music edu packscholar #x 1999 7 6639.040000 2653.55 4219.52 10071.22 +Music amalgscholar #x 1999 4 6719.304167 2739.33 3690.54 3872.98 +Men importoimporto #x 1999 3 6610.034167 2645.24 3661.14 4282.01 +Music exportischolar #x 1999 2 7043.051667 3115.83 4457.95 5258.95 +Men edu packimporto #x 1999 3 7386.333333 3488.01 3347.65 3329.74 +Shoes exportiedu pack #x 1999 3 7255.790000 3411.07 4194.64 3624.85 +Women exportiamalg #x 1999 2 5646.671667 1809.52 4198.70 2172.85 +Music importoscholar #x 1999 6 6279.081667 2456.98 4361.44 4256.24 +Children importoexporti #x 1999 7 6786.750000 2978.82 3942.59 7809.22 +Music exportischolar #x 1999 2 7041.705833 3245.77 3608.31 4127.40 +Shoes importoedu pack #x 1999 2 6864.320833 3104.48 3135.52 3606.28 +Shoes importoedu pack #x 1999 1 6864.320833 3135.52 14580.30 3104.48 +Children edu packexporti #x 1999 5 6511.800833 2785.92 3956.90 3906.63 +Women amalgamalg #x 1999 2 6480.683333 2769.69 3353.67 3876.51 +Men exportiimporto #x 1999 5 7146.240000 3440.57 3561.11 3971.13 +Women importoamalg #x 1999 3 6512.794167 2808.28 3789.51 4335.27 +Men amalgimporto #x 1999 2 6720.550000 3018.90 4328.03 3810.74 +Shoes exportiedu pack #x 1999 7 7255.790000 3557.87 4937.98 9496.49 +Music exportischolar #x 1999 5 6791.260833 3096.02 3918.04 3801.90 +Men exportiimporto #x 1999 1 7343.719167 3652.72 13689.13 3984.13 +Men importoimporto #x 1999 5 6707.315833 3030.18 4977.33 4620.75 +Children exportiexporti #x 1999 2 6386.880833 2717.07 4809.11 3355.48 +Men amalgimporto #x 1999 2 7098.680833 3440.27 5293.69 3526.45 +Men importoimporto #x 1999 5 6610.034167 2954.71 4282.01 3166.43 +Shoes exportiedu pack #x 1999 4 7255.790000 3624.85 3411.07 5169.09 +Men exportiimporto #x 1999 5 7343.719167 3727.75 3729.62 4580.93 +Men exportiimporto #x 1999 4 7343.719167 3729.62 4033.37 3727.75 +Music edu packscholar #x 1999 2 6489.175000 2875.98 4299.82 4028.97 +Men edu packimporto #x 1999 1 7202.242500 3614.07 15582.63 4234.79 +Music importoscholar #x 1999 2 5816.271667 2229.79 2919.29 4298.41 +Men exportiimporto #x 1999 4 7146.240000 3561.11 5091.17 3440.57 +Shoes exportiedu pack #x 1999 7 7073.462500 3493.41 4534.31 8701.59 +Music exportischolar #x 1999 3 6791.260833 3218.13 3847.87 3918.04 +Shoes edu packedu pack #x 1999 4 6203.331667 2631.02 4424.18 4186.85 +Men amalgimporto #x 1999 3 7098.680833 3526.45 3440.27 2965.42 +Men edu packimporto #x 1999 3 7202.242500 3639.93 4234.79 4016.25 +Children amalgexporti #x 1999 2 6364.467500 2825.09 4111.08 3330.83 +Shoes edu packedu pack #x 1999 2 6464.239167 2928.99 4233.86 3840.97 +Shoes amalgedu pack #x 1999 4 6493.071667 2962.20 3443.20 4212.60 +Music importoscholar #x 1999 4 5707.844167 2179.41 3789.16 4317.53 +Shoes exportiedu pack #x 1999 1 7416.141667 3892.54 14170.68 4654.22 +Women importoamalg #x 1999 5 6512.794167 2991.07 4335.27 4624.86 +Music exportischolar #x 1999 4 7043.051667 3521.99 5258.95 4135.21 +Women edu packamalg #x 1999 2 6354.045833 2836.23 3719.67 3527.07 +Music amalgscholar #x 1999 3 6123.475000 2617.39 3080.43 4919.93 +Shoes amalgedu pack #x 1999 7 6674.896667 3178.12 3342.98 8050.81 +Men amalgimporto #x 1999 5 6618.534167 3127.28 4291.66 4669.62 +Women amalgamalg #x 1999 6 6874.250000 3387.53 4798.69 4329.48 +Women exportiamalg #x 1999 3 5646.671667 2172.85 1809.52 4461.31 +Children edu packexporti #x 1999 2 6112.954167 2641.54 3567.58 3196.45 +Children amalgexporti #x 1999 5 6294.100833 2834.48 3317.60 3803.79 +Women edu packamalg #x 1999 5 6027.880000 2575.81 2750.96 4459.01 +Music exportischolar #x 1999 6 7041.705833 3589.97 4134.03 4892.26 +Music exportischolar #x 1999 4 7041.705833 3593.86 4127.40 4134.03 +Men importoimporto #x 1999 6 6610.034167 3166.43 2954.71 3673.39 +Music exportischolar #x 1999 1 7041.705833 3608.31 15046.54 3245.77 +Music edu packscholar #x 1999 2 6602.385000 3173.44 3434.91 3929.40 +Music amalgscholar #x 1999 6 6123.475000 2699.75 4038.47 3330.87 +Children importoexporti #x 1999 4 6786.750000 3366.25 3847.60 4259.57 +Men edu packimporto #x 1999 1 7230.493333 3811.64 14668.93 4497.31 +Shoes importoedu pack #x 1999 5 6864.320833 3449.62 3869.15 3531.93 +Children edu packexporti #x 1999 2 6739.498333 3328.01 4986.50 3623.32 +Children importoexporti #x 1999 1 6786.750000 3376.55 12504.93 5018.22 +Children edu packexporti #x 1999 7 6112.954167 2711.04 3254.04 9465.10 +Shoes importoedu pack #x 1999 3 6588.741667 3187.25 4283.21 3212.76 +Men importoimporto #x 1999 3 6702.415000 3310.55 3981.06 4901.56 +Men edu packimporto #x 1999 1 7386.333333 4007.40 14005.45 3347.65 +Shoes importoedu pack #x 1999 4 6588.741667 3212.76 3187.25 3974.78 +Shoes edu packedu pack #x 1999 6 6203.331667 2835.78 4186.85 3192.53 +Men exportiimporto #x 1999 2 7343.719167 3984.13 3652.72 4033.37 +Men amalgimporto #x 1999 4 6720.550000 3364.32 3810.74 4333.58 +Children edu packexporti #x 1999 4 6739.498333 3389.03 3623.32 3605.25 +Shoes importoedu pack #x 1999 6 6864.320833 3531.93 3449.62 4414.17 +Shoes amalgedu pack #x 1999 6 6674.896667 3342.98 4458.26 3178.12 +Children edu packexporti #x 1999 2 6511.800833 3185.28 3581.75 3410.75 +Children amalgexporti #x 1999 4 6854.405833 3541.62 3854.33 3938.42 +Men exportiimporto #x 1999 3 7343.719167 4033.37 3984.13 3729.62 +Men amalgimporto #x 1999 3 6618.534167 3313.62 4044.36 4291.66 +Shoes exportiedu pack #x 1999 7 7416.141667 4121.24 4239.08 8658.42 +Women importoamalg #x 1999 6 6395.326667 3102.55 4234.22 3650.03 +Children importoexporti #x 1999 3 6577.143333 3291.07 3773.61 2152.15 +Women edu packamalg #x 1999 4 6027.880000 2750.96 3199.35 2575.81 +Music amalgscholar #x 1999 3 6662.669167 3386.25 1961.57 4799.18 +Men amalgimporto #x 1999 6 7098.680833 3834.90 4923.53 4115.57 +Shoes importoedu pack #x 1999 3 6864.320833 3606.28 3104.48 3869.15 +Music exportischolar #x 1999 6 7043.051667 3793.48 4135.21 5006.69 +Shoes edu packedu pack #x 1999 1 6711.753333 3473.10 15060.83 4085.86 +Men exportiimporto #x 1999 1 7419.459167 4188.88 16358.86 4366.77 +Women amalgamalg #x 1999 2 6362.709167 3137.03 4180.91 3181.26 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q5a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q5a.sql.out new file mode 100644 index 0000000000000..600c51b691b35 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q5a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 115594110.84 2284876.41 -30784735.42 +catalog channel NULL 39386111.98 835889.43 -4413707.25 +catalog channel catalog_pageAAAAAAAAAABAAAAA 142709.95 0.00 -33829.11 +catalog channel catalog_pageAAAAAAAAAADAAAAA 78358.50 0.00 -9578.35 +catalog channel catalog_pageAAAAAAAAABAAAAAA 0.00 4176.90 -492.29 +catalog channel catalog_pageAAAAAAAAABBAAAAA 71027.83 0.00 -27264.22 +catalog channel catalog_pageAAAAAAAAABDAAAAA 93670.35 0.00 -3768.98 +catalog channel catalog_pageAAAAAAAAACAAAAAA 0.00 1776.17 -1252.65 +catalog channel catalog_pageAAAAAAAAACBAAAAA 112153.16 0.00 -19004.12 +catalog channel catalog_pageAAAAAAAAACDAAAAA 102298.11 0.00 -14768.21 +catalog channel catalog_pageAAAAAAAAADAAAAAA 0.00 8773.82 -4372.46 +catalog channel catalog_pageAAAAAAAAADBAAAAA 23722.97 0.00 -22193.13 +catalog channel catalog_pageAAAAAAAAADDAAAAA 47126.10 0.00 -6735.40 +catalog channel catalog_pageAAAAAAAAAEAAAAAA 0.00 6234.69 -2690.18 +catalog channel catalog_pageAAAAAAAAAEBAAAAA 45880.56 679.36 -16578.99 +catalog channel catalog_pageAAAAAAAAAEDAAAAA 66004.24 0.00 6074.29 +catalog channel catalog_pageAAAAAAAAAFAAAAAA 0.00 463.68 -1521.36 +catalog channel catalog_pageAAAAAAAAAFDAAAAA 88022.65 0.00 -769.81 +catalog channel catalog_pageAAAAAAAAAGAAAAAA 0.00 8947.80 -2922.65 +catalog channel catalog_pageAAAAAAAAAGDAAAAA 124552.05 0.00 24880.33 +catalog channel catalog_pageAAAAAAAAAHAAAAAA 188643.97 0.00 -33045.93 +catalog channel catalog_pageAAAAAAAAAIAAAAAA 219950.84 518.83 -1089.03 +catalog channel catalog_pageAAAAAAAAAJAAAAAA 196607.25 7494.81 -41407.88 +catalog channel catalog_pageAAAAAAAAAJCAAAAA 23556.72 0.00 -5599.24 +catalog channel catalog_pageAAAAAAAAAKAAAAAA 155095.31 0.00 -24248.39 +catalog channel catalog_pageAAAAAAAAAKCAAAAA 13054.83 0.00 -528.84 +catalog channel catalog_pageAAAAAAAAALAAAAAA 114177.20 1866.02 -48127.76 +catalog channel catalog_pageAAAAAAAAALCAAAAA 10602.13 0.00 -3956.02 +catalog channel catalog_pageAAAAAAAAAMAAAAAA 180993.77 144.00 -26262.58 +catalog channel catalog_pageAAAAAAAAAMCAAAAA 29191.46 0.00 -8019.34 +catalog channel catalog_pageAAAAAAAAANAAAAAA 202640.83 0.00 14603.23 +catalog channel catalog_pageAAAAAAAAANBAAAAA 0.00 3103.80 -1162.99 +catalog channel catalog_pageAAAAAAAAANCAAAAA 14627.25 61.80 1219.16 +catalog channel catalog_pageAAAAAAAAAOAAAAAA 49178.54 0.00 -27661.86 +catalog channel catalog_pageAAAAAAAAAOCAAAAA 9621.16 0.00 -4213.22 +catalog channel catalog_pageAAAAAAAAAPAAAAAA 142216.57 0.00 21113.94 +catalog channel catalog_pageAAAAAAAAAPBAAAAA 0.00 4876.83 -2903.30 +catalog channel catalog_pageAAAAAAAAAPCAAAAA 17230.02 689.60 -1816.99 +catalog channel catalog_pageAAAAAAAABAAAAAAA 0.00 1902.91 -1516.52 +catalog channel catalog_pageAAAAAAAABABAAAAA 100018.02 0.00 -9370.45 +catalog channel catalog_pageAAAAAAAABADAAAAA 115622.82 0.00 12126.64 +catalog channel catalog_pageAAAAAAAABBAAAAAA 0.00 1786.21 -891.28 +catalog channel catalog_pageAAAAAAAABBBAAAAA 76388.45 1153.20 2393.80 +catalog channel catalog_pageAAAAAAAABBCAAAAA 0.00 831.90 -1077.03 +catalog channel catalog_pageAAAAAAAABBDAAAAA 47684.96 0.00 -8371.50 +catalog channel catalog_pageAAAAAAAABCAAAAAA 0.00 4371.58 -4222.90 +catalog channel catalog_pageAAAAAAAABCBAAAAA 93859.97 1753.50 -1196.59 +catalog channel catalog_pageAAAAAAAABCDAAAAA 73347.51 0.00 -11342.92 +catalog channel catalog_pageAAAAAAAABDAAAAAA 0.00 3474.90 -1008.22 +catalog channel catalog_pageAAAAAAAABDBAAAAA 42173.47 0.00 -26341.79 +catalog channel catalog_pageAAAAAAAABDCAAAAA 0.00 305.46 -585.35 +catalog channel catalog_pageAAAAAAAABDDAAAAA 83810.14 0.00 -1482.89 +catalog channel catalog_pageAAAAAAAABEAAAAAA 0.00 347.39 -197.16 +catalog channel catalog_pageAAAAAAAABEBAAAAA 97527.42 0.00 2333.17 +catalog channel catalog_pageAAAAAAAABEDAAAAA 60506.22 0.00 -1212.87 +catalog channel catalog_pageAAAAAAAABFAAAAAA 0.00 6528.33 -2187.98 +catalog channel catalog_pageAAAAAAAABFDAAAAA 68737.67 0.00 -11721.44 +catalog channel catalog_pageAAAAAAAABHAAAAAA 199767.35 1044.68 -6811.92 +catalog channel catalog_pageAAAAAAAABIAAAAAA 205306.84 411.12 2555.23 +catalog channel catalog_pageAAAAAAAABICAAAAA 0.00 68.02 -99.54 +catalog channel catalog_pageAAAAAAAABJAAAAAA 187825.12 230.88 -21543.64 +catalog channel catalog_pageAAAAAAAABJBAAAAA 0.00 727.89 -209.17 +catalog channel catalog_pageAAAAAAAABJCAAAAA 14241.32 0.00 4283.83 +catalog channel catalog_pageAAAAAAAABKAAAAAA 215820.15 1121.48 -7030.88 +catalog channel catalog_pageAAAAAAAABKBAAAAA 0.00 3063.15 -3543.44 +catalog channel catalog_pageAAAAAAAABKCAAAAA 1871.32 0.00 -2063.64 +catalog channel catalog_pageAAAAAAAABLAAAAAA 171643.10 779.60 -15082.45 +catalog channel catalog_pageAAAAAAAABLCAAAAA 8855.03 0.00 3013.67 +catalog channel catalog_pageAAAAAAAABMAAAAAA 202476.57 1954.44 10034.15 +catalog channel catalog_pageAAAAAAAABMCAAAAA 13837.48 0.00 6008.90 +catalog channel catalog_pageAAAAAAAABNAAAAAA 197464.73 1397.02 -34977.14 +catalog channel catalog_pageAAAAAAAABNCAAAAA 14801.05 0.00 -6910.60 +catalog channel catalog_pageAAAAAAAABOAAAAAA 98871.66 0.00 -6493.20 +catalog channel catalog_pageAAAAAAAABOCAAAAA 15216.80 0.00 -5722.57 +catalog channel catalog_pageAAAAAAAABPAAAAAA 99238.38 100.20 -6353.28 +catalog channel catalog_pageAAAAAAAABPCAAAAA 26783.54 0.00 4129.36 +catalog channel catalog_pageAAAAAAAACAAAAAAA 0.00 8072.86 -3459.85 +catalog channel catalog_pageAAAAAAAACABAAAAA 139938.46 0.00 8612.80 +catalog channel catalog_pageAAAAAAAACADAAAAA 36613.51 0.00 -9705.44 +catalog channel catalog_pageAAAAAAAACBAAAAAA 0.00 9373.27 -7982.93 +catalog channel catalog_pageAAAAAAAACBBAAAAA 111531.45 55.38 -15921.86 +catalog channel catalog_pageAAAAAAAACBDAAAAA 103545.90 0.00 -168.10 +catalog channel catalog_pageAAAAAAAACCAAAAAA 0.00 3879.26 -1783.40 +catalog channel catalog_pageAAAAAAAACCBAAAAA 78290.87 0.00 -12841.84 +catalog channel catalog_pageAAAAAAAACCDAAAAA 37841.79 0.00 -26568.14 +catalog channel catalog_pageAAAAAAAACDAAAAAA 0.00 923.32 -1166.75 +catalog channel catalog_pageAAAAAAAACDBAAAAA 113988.86 212.31 -9856.53 +catalog channel catalog_pageAAAAAAAACDDAAAAA 64516.71 0.00 -27577.04 +catalog channel catalog_pageAAAAAAAACEAAAAAA 0.00 2955.12 -1700.94 +catalog channel catalog_pageAAAAAAAACEBAAAAA 91555.04 0.00 -23392.02 +catalog channel catalog_pageAAAAAAAACEDAAAAA 84993.13 0.00 -24259.30 +catalog channel catalog_pageAAAAAAAACFAAAAAA 0.00 6507.25 -3250.26 +catalog channel catalog_pageAAAAAAAACFDAAAAA 73749.97 0.00 1468.23 +catalog channel catalog_pageAAAAAAAACGAAAAAA 0.00 8814.60 -3790.45 +catalog channel catalog_pageAAAAAAAACHAAAAAA 157602.51 907.12 -50822.23 +catalog channel catalog_pageAAAAAAAACHCAAAAA 0.00 59.58 -63.15 +catalog channel catalog_pageAAAAAAAACIAAAAAA 132342.50 4686.24 -45195.01 +catalog channel catalog_pageAAAAAAAACIBAAAAA 0.00 6029.56 -3462.80 +catalog channel catalog_pageAAAAAAAACICAAAAA 0.00 530.41 -535.89 +catalog channel catalog_pageAAAAAAAACJAAAAAA 193536.42 2487.88 -29573.52 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q6.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q6.sql.out new file mode 100644 index 0000000000000..86d2d430218f7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q6.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +VT 10 +MA 11 +NH 11 +NJ 14 +NV 16 +OR 16 +WY 16 +AK 20 +ME 22 +MD 26 +WA 29 +ID 31 +NM 31 +UT 36 +ND 38 +SC 41 +SD 43 +WV 44 +CA 52 +FL 53 +LA 56 +PA 57 +NY 59 +AR 61 +CO 61 +WI 61 +MT 66 +MS 68 +OK 68 +MN 71 +OH 75 +MO 79 +IL 80 +AL 81 +NC 81 +IA 82 +MI 84 +KS 86 +NE 86 +NULL 88 +IN 94 +TN 108 +KY 110 +VA 128 +GA 138 +TX 250 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q64.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q64.sql.out new file mode 100644 index 0000000000000..a803af3a52c21 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q64.sql.out @@ -0,0 +1,14 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +ablepricallyantiought ation 35709 996 NULL Bridgeport 65817 752 Lakeview Lincoln Friendship 74536 1999 1 15.78 24.93 0.00 17.09 26.31 0.00 2000 1 +ablepricallyantiought ation 35709 71 River River Friendship 34536 NULL NULL Newport NULL 1999 1 22.60 38.87 0.00 17.09 26.31 0.00 2000 1 +ablepricallyantiought bar 31904 128 East Franklin 19101 990 2nd Friendship 94536 1999 1 54.76 78.30 0.00 15.80 23.54 0.00 2000 1 +ationbarpri ation 35709 362 Central Ridge Oakland 69843 666 13th Ridge Shiloh 29275 1999 1 74.00 95.46 0.00 11.32 20.94 0.00 2000 1 +ationbarpri ese 31904 759 Elm Pine Belmont 20191 35 Madison Waterloo 31675 1999 1 12.92 22.22 0.00 24.15 36.70 0.00 2000 1 +ationbarpri ese 31904 759 Elm Pine Belmont 20191 35 Madison Waterloo 31675 1999 1 12.92 22.22 0.00 83.87 147.61 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 12.02 12.74 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 28.08 38.18 0.00 2000 1 +ationbarpri ought 31904 754 NULL NULL 65804 897 8th Ashland 54244 1999 1 74.70 90.38 0.00 56.60 63.39 0.00 2000 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q67a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q67a.sql.out new file mode 100644 index 0000000000000..2c460414e8623 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q67a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL NULL NULL NULL NULL NULL NULL 681149.47 5 +NULL NULL NULL NULL NULL NULL NULL NULL 1474997.55 4 +NULL NULL NULL NULL NULL NULL NULL NULL 1577864.42 3 +NULL NULL NULL NULL NULL NULL NULL NULL 2536204.81 2 +NULL NULL NULL NULL NULL NULL NULL NULL 999668082.59 1 +NULL NULL NULL NULL 2001 NULL NULL NULL 681149.47 5 +NULL NULL NULL NULL 2001 1 NULL NULL 111437.42 32 +NULL NULL NULL NULL 2001 1 1 NULL 36235.73 90 +NULL NULL NULL NULL 2001 1 3 NULL 49252.88 77 +NULL NULL NULL NULL 2001 2 NULL NULL 78346.18 63 +NULL NULL NULL NULL 2001 2 6 NULL 34840.48 96 +NULL NULL NULL NULL 2001 3 NULL NULL 141691.87 12 +NULL NULL NULL NULL 2001 3 7 NULL 31988.52 100 +NULL NULL NULL NULL 2001 3 8 NULL 57778.28 69 +NULL NULL NULL NULL 2001 3 9 NULL 51925.07 73 +NULL NULL NULL NULL 2001 4 NULL NULL 349674.00 7 +NULL NULL NULL NULL 2001 4 10 NULL 89521.66 52 +NULL NULL NULL NULL 2001 4 11 NULL 107989.95 35 +NULL NULL NULL NULL 2001 4 11 AAAAAAAAHAAAAAAA 38673.65 85 +NULL NULL NULL NULL 2001 4 12 NULL 152162.39 11 +NULL NULL NULL NULL 2001 4 12 AAAAAAAACAAAAAAA 36936.71 88 +NULL NULL NULL NULL 2001 4 12 AAAAAAAAKAAAAAAA 35885.68 92 +NULL NULL NULL ationesebareing NULL NULL NULL NULL 130971.51 13 +NULL NULL NULL ationesebareing 2001 NULL NULL NULL 130971.51 13 +NULL NULL NULL ationesebareing 2001 3 NULL NULL 49334.88 76 +NULL NULL NULL ationesebareing 2001 4 NULL NULL 36033.41 91 +NULL NULL NULL n stcallyn stn st NULL NULL NULL NULL 110926.60 33 +NULL NULL NULL n stcallyn stn st 2001 NULL NULL NULL 110926.60 33 +NULL NULL NULL n stcallyn stn st 2001 3 NULL NULL 33118.05 98 +NULL NULL NULL n stcallyn stn st 2001 4 NULL NULL 41603.21 83 +NULL NULL NULL n stought NULL NULL NULL NULL 157465.43 9 +NULL NULL NULL n stought 2001 NULL NULL NULL 157465.43 9 +NULL NULL NULL n stought 2001 3 NULL NULL 46371.14 78 +NULL NULL NULL n stought 2001 4 NULL NULL 65619.67 66 +NULL NULL NULL n stought 2001 4 11 NULL 45528.69 79 +NULL NULL NULL oughtableantiable NULL NULL NULL NULL 103187.19 40 +NULL NULL NULL oughtableantiable 2001 NULL NULL NULL 103187.19 40 +NULL NULL NULL oughtableantiable 2001 4 NULL NULL 42408.80 82 +NULL NULL NULL oughtablen stationought NULL NULL NULL NULL 86348.22 57 +NULL NULL NULL oughtablen stationought 2001 NULL NULL NULL 86348.22 57 +NULL NULL NULL oughtablen stationought 2001 4 NULL NULL 35714.86 93 +NULL NULL NULL oughtesebaration NULL NULL NULL NULL 76325.38 64 +NULL NULL NULL oughtesebaration 2001 NULL NULL NULL 76325.38 64 +NULL NULL NULL oughtesebaration 2001 4 NULL NULL 36610.75 89 +NULL NULL NULL oughteseoughtation NULL NULL NULL NULL 128623.75 19 +NULL NULL NULL oughteseoughtation 2001 NULL NULL NULL 128623.75 19 +NULL NULL NULL oughteseoughtation 2001 3 NULL NULL 35495.82 94 +NULL NULL NULL oughteseoughtation 2001 4 NULL NULL 57203.17 70 +NULL NULL importoamalg #x NULL NULL NULL NULL NULL 102866.87 42 +NULL NULL importoamalg #x NULL NULL NULL NULL NULL 102866.87 42 +NULL NULL importoamalg #x NULL 2001 NULL NULL NULL 102866.87 42 +NULL NULL importoamalg #x NULL 2001 4 NULL NULL 51717.30 74 +NULL accessories NULL NULL NULL NULL NULL NULL 98994.39 48 +NULL accessories NULL NULL NULL NULL NULL NULL 98994.39 48 +NULL accessories NULL prin stn stoughtought NULL NULL NULL NULL 98994.39 48 +NULL accessories NULL prin stn stoughtought 2001 NULL NULL NULL 98994.39 48 +NULL accessories NULL prin stn stoughtought 2001 3 NULL NULL 37246.83 87 +NULL athletic NULL NULL NULL NULL NULL NULL 122120.16 21 +NULL athletic NULL NULL NULL NULL NULL NULL 122120.16 21 +NULL athletic NULL NULL NULL NULL NULL NULL 122120.16 21 +NULL athletic NULL NULL 2001 NULL NULL NULL 122120.16 21 +NULL athletic NULL NULL 2001 4 NULL NULL 63278.49 67 +NULL athletic NULL NULL 2001 4 11 NULL 38216.51 86 +NULL baseball NULL NULL NULL NULL NULL NULL 105806.96 36 +NULL baseball NULL NULL NULL NULL NULL NULL 105806.96 36 +NULL baseball NULL NULL NULL NULL NULL NULL 105806.96 36 +NULL baseball NULL NULL 2001 NULL NULL NULL 105806.96 36 +NULL baseball NULL NULL 2001 4 NULL NULL 58206.33 68 +NULL country NULL NULL NULL NULL NULL NULL 89164.69 53 +NULL country importoscholar #x NULL NULL NULL NULL NULL 89164.69 53 +NULL country importoscholar #x NULL NULL NULL NULL NULL 89164.69 53 +NULL country importoscholar #x NULL 2001 NULL NULL NULL 89164.69 53 +NULL country importoscholar #x NULL 2001 4 NULL NULL 55674.96 71 +NULL dresses NULL NULL NULL NULL NULL NULL 116126.30 25 +NULL dresses NULL NULL NULL NULL NULL NULL 217647.20 8 +NULL dresses NULL oughteingantieing NULL NULL NULL NULL 116126.30 25 +NULL dresses NULL oughteingantieing 2001 NULL NULL NULL 116126.30 25 +NULL dresses NULL oughteingantieing 2001 3 NULL NULL 35421.52 95 +NULL dresses NULL oughteingantieing 2001 4 NULL NULL 53283.33 72 +NULL dresses amalgamalg #x NULL NULL NULL NULL NULL 101520.90 45 +NULL dresses amalgamalg #x NULL NULL NULL NULL NULL 101520.90 45 +NULL dresses amalgamalg #x NULL 2001 NULL NULL NULL 101520.90 45 +NULL dresses amalgamalg #x NULL 2001 3 NULL NULL 32017.28 99 +NULL dresses amalgamalg #x NULL 2001 4 NULL NULL 34451.25 97 +NULL infants NULL NULL NULL NULL NULL NULL 128750.23 15 +NULL infants importoexporti #x NULL NULL NULL NULL NULL 128750.23 15 +NULL infants importoexporti #x antiationeseese NULL NULL NULL NULL 128750.23 15 +NULL infants importoexporti #x antiationeseese 2001 NULL NULL NULL 128750.23 15 +NULL infants importoexporti #x antiationeseese 2001 3 NULL NULL 40113.56 84 +NULL infants importoexporti #x antiationeseese 2001 4 NULL NULL 50724.87 75 +NULL mens NULL NULL NULL NULL NULL NULL 113853.60 28 +NULL mens importoedu pack #x NULL NULL NULL NULL NULL 113853.60 28 +NULL mens importoedu pack #x oughtablen steseought NULL NULL NULL NULL 113853.60 28 +NULL mens importoedu pack #x oughtablen steseought 2001 NULL NULL NULL 113853.60 28 +NULL mens importoedu pack #x oughtablen steseought 2001 3 NULL NULL 42447.63 81 +NULL mens importoedu pack #x oughtablen steseought 2001 4 NULL NULL 43089.74 80 +NULL shirts NULL NULL NULL NULL NULL NULL 82003.16 59 +NULL shirts NULL NULL NULL NULL NULL NULL 82003.16 59 +NULL shirts NULL oughtcallyeseantiought NULL NULL NULL NULL 82003.16 59 +NULL shirts NULL oughtcallyeseantiought 2001 NULL NULL NULL 82003.16 59 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q70a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q70a.sql.out new file mode 100644 index 0000000000000..067238b4407b6 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q70a.sql.out @@ -0,0 +1,8 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +-435656177.80 NULL NULL 2 1 +-435656177.80 TN NULL 1 1 +-435656177.80 TN Williamson County 0 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q72.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q72.sql.out new file mode 100644 index 0000000000000..f4050e94d4c53 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q72.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +Again perfect sons used to put always. Europea Significantly 5305 2 2 2 +Ago new arguments accept previously european parents; fo Selective, 5313 2 2 2 +Difficult, rapid sizes say so; initial banks stress high single sports; prisoners used to think likely firms. Good, current services must take human, precise persons. Signals m Selective, 5322 2 2 2 +Eventually soft issues will see pp.; possible children pay completely future tasks. Only women will not rehearse more old parts; different movements sponsor most. Please political allowance Significantly 5273 2 2 2 +Happy, nuclear obligations should not leave little payments. About able relationships impress thus only original a Significantly 5313 2 2 2 +Healthy, delighted conclusions may offer experienced condi Significantly 5305 2 2 2 +Hitherto certain kinds evade also by a months. Poor points might make even just selective passengers. Old, general qualities could overcome over; recent variables might s Just good amou 5280 2 2 2 +Ideas total sadly. International members contribute perhaps lucky cells. Texts would acknowled Selective, 5280 2 2 2 +Important audiences shall murder. Selective, 5322 2 2 2 +In order suitable conditions used to eat more on a americans. As open rates shall go sometimes big authorities. Tickets respond for example pregnant, good women. Banks could exploit also full, old cr Matches produce 5319 2 2 2 +Little, num Matches produce 5322 2 2 2 +National, l Operations 5318 2 2 2 +Obvious, present cells may not define appointed, military boys. Answers can get little. Over there comparative days become in a police. Particularly divine prop Just good amou 5322 2 2 2 +Outside, useful animals find again also british decisions; now small attitudes shall n Significantly 5322 2 2 2 +Particular, british wa Operations 5313 2 2 2 +Parts will recruit si Selective, 5322 2 2 2 +Rational, given goods would stay just equal materials. Very alternative months might not buy voc Just good amou 5319 2 2 2 +Royal, military notions will not find very very wet acids. Funny actions take western, remaining homes. Great patients will replace simply. Signs can think equivalent reasons. Campaigns Matches produce 5313 2 2 2 +Short companies get that is for a days. Early, special hands might keep to the women. Present blocks co-ordinate so sure results. Seasons lend still recent friends. Dead Just good amou 5273 2 2 2 +Short companies get that is for a days. Early, special hands might keep to the women. Present blocks co-ordinate so sure results. Seasons lend still recent friends. Dead Selective, 5273 2 2 2 +Specialist, identical factors should not answer so main shares. Sales might reduce then free hours. Ethic Just good amou 5306 2 2 2 +Then sp Selective, 5308 2 2 2 +NULL Just good amou 5297 1 1 1 +NULL Matches produce 5282 1 1 1 +NULL Operations 5308 1 1 1 +NULL Selective, 5318 1 1 1 +A bit important Significantly 5284 1 1 1 +A bit liable flowers change also writings. Currently soviet ministers come. Hotels telephone before aggressive, economic eyes. Blue changes improve. Overal Operations 5317 1 1 1 +A bit liable flowers change also writings. Currently soviet ministers come. Hotels telephone before aggressive, economic eyes. Blue changes improve. Overal Selective, 5317 1 1 1 +A little local letters think over like a children; nevertheless particular powers damage now suddenly absent prote Significantly 5317 1 1 1 +Able systems merge from a areas. Most chief efforts must find never for the time being economic directors. Activities sit there. Available polic Selective, 5305 1 1 1 +Able troubles dust into the styles. Independent feet kill wounds. Fundamental months should exploit arms. Massive years read only modern courses; twin forms shall become products. Even h Matches produce 5309 1 1 1 +Able troubles dust into the styles. Independent feet kill wounds. Fundamental months should exploit arms. Massive years read only modern courses; twin forms shall become products. Even h Operations 5309 1 1 1 +Able, active jobs might not play upstairs. Electoral crimes could not worry for the solutions. Wholly capitalist effects would not get Matches produce 5291 1 1 1 +Able, initial men cannot assume then rational, new references; shares could support physical, internati Operations 5304 1 1 1 +Able, reasonable standards make forward. Strategic, Just good amou 5319 1 1 1 +About military programmes identify all in a thousands; able sentences mean also flats. Branches know secrets; right, increased interactions tour a little. High, lov Selective, 5299 1 1 1 +About supreme days tell then for a consequences. Ill items force meals; years may not mean quite social structures. Goals Significantly 5311 1 1 1 +Above, new groups will not like much local bodies. However traditional sessions can walk slowly big, young aspects. Quite close companies ought to take in a rules. Leaders must not like of cou Just good amou 5314 1 1 1 +Above, new groups will not like much local bodies. However traditional sessions can walk slowly big, young aspects. Quite close companies ought to take in a rules. Leaders must not like of cou Matches produce 5314 1 1 1 +Above, new groups will not like much local bodies. However traditional sessions can walk slowly big, young aspects. Quite close companies ought to take in a rules. Leaders must not like of cou Matches produce 5315 1 1 1 +Absolute Matches produce 5322 1 1 1 +Absolutely angry odds put strongly. Telecommunications help only recent, Selective, 5316 1 1 1 +Ac Significantly 5306 1 1 1 +Accessible, likel Matches produce 5301 1 1 1 +Accessible, likel Selective, 5301 1 1 1 +Accessible, old walls profit here. Wars form therefore as effective servants. Secrets could not feel meanwhile regional theories. Perfect, new service Operations 5273 1 1 1 +Accidents can include below other, marginal plans. Comparable, welsh exceptions argue most as usual physical claims. Certain months may smell far from in a cases. Active seconds used to restore t Matches produce 5304 1 1 1 +Accidents fly bet Matches produce 5321 1 1 1 +Accounts rank only high plans. Days sho Just good amou 5302 1 1 1 +Accurately economic workers play clearly. Deliberately other stands recapture social, cultural prices. Full paths used to make twice alw Matches produce 5301 1 1 1 +Actions must not compare so economi Matches produce 5320 1 1 1 +Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Significantly 5289 1 1 1 +Actual arrangements should introduce never in a unions. Ultimately d Just good amou 5273 1 1 1 +Actual, possible sides employ here future hands. Powerful intervals ought to respond new, particular marks. Appointed, spiritual accidents sustain but modern, coming findings. Male, national year Selective, 5304 1 1 1 +Actually subtle subjects mark as tories. Yet possible areas Just good amou 5321 1 1 1 +Actually subtle subjects mark as tories. Yet possible areas Matches produce 5320 1 1 1 +Acutely possible kilometres cannot trim fully justly original visitors. Owners can transport from the connections. Then controversial girls might tell yet more big kinds. More typical houses g Just good amou 5272 1 1 1 +Acutely possible kilometres cannot trim fully justly original visitors. Owners can transport from the connections. Then controversial girls might tell yet more big kinds. More typical houses g Selective, 5318 1 1 1 +Additional companies visit. Grey opportunities may not look numbers. Entire, british models assist also great quarters. Little males show Just good amou 5284 1 1 1 +Additional companies visit. Grey opportunities may not look numbers. Entire, british models assist also great quarters. Little males show Selective, 5284 1 1 1 +Additional figures consult relationships. Sole addresses convict right, Operations 5322 1 1 1 +Advantages emerge moves; special, expected operations pass etc natural preferences; very posit Selective, 5313 1 1 1 +Afterwards defensive standards answer just almost informal officers. Now constant rights shall hear courses. Signs go on a budgets Just good amou 5280 1 1 1 +Afterwards oth Just good amou 5277 1 1 1 +Afterwards oth Selective, 5277 1 1 1 +Afterwards rich options go unlikely, welsh elections. Just gentle authors must not provi Operations 5285 1 1 1 +Afterwards written skills influence; english, level departments like just. Really legal rocks would Just good amou 5300 1 1 1 +Again Just good amou 5319 1 1 1 +Again brief things should remember only in a patients. Deals reply soon other points. Increasingly religious times necessitate farther troops. Both added programmes must come wonderfully solid pupi Matches produce 5308 1 1 1 +Again brief things should remember only in a patients. Deals reply soon other points. Increasingly religious times necessitate farther troops. Both added programmes must come wonderfully solid pupi Operations 5308 1 1 1 +Again brief things should remember only in a patients. Deals reply soon other points. Increasingly religious times necessitate farther troops. Both added programmes must come wonderfully solid pupi Selective, 5308 1 1 1 +Again new teeth heat delicately. Just future officers Just good amou 5319 1 1 1 +Again new teeth heat delicately. Just future officers Operations 5294 1 1 1 +Again new teeth heat delicately. Just future officers Significantly 5294 1 1 1 +Again old police could work in the skills. Points announce agents. Pieces conform slowly to a hea Significantly 5307 1 1 1 +Again scottish women shall ag Significantly 5308 1 1 1 +Agencies will not move criminal issues. Years mean very largel Selective, 5305 1 1 1 +Agents invest often things. French cars ought to get locally distinctive, local powers. More american entries compensate only Operations 5317 1 1 1 +Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Just good amou 5317 1 1 1 +Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Operations 5304 1 1 1 +Ago interested doctors meet really fair, cold minds. Fine children understand original procedures. So other services ought to Operations 5303 1 1 1 +Ago interested doctors meet really fair, cold minds. Fine children understand original procedures. So other services ought to Significantly 5303 1 1 1 +Ago new studies shall not apply of course small forces. Dead parts used to point on a students. Then other students should pay only Matches produce 5307 1 1 1 +Ago sexual courts may attract. Important, alone observations expect. New, available ways represent years. Excell Matches produce 5301 1 1 1 +Ahead national cir Matches produce 5320 1 1 1 +All attractive ways develop originally lucky sites. New, single sounds might excuse enough senior savings. Other bacteria live across a concerns. Dark minutes s Significantly 5316 1 1 1 +All capital bacteria make jobs. Again appropriate eyes may not leave others. There fixed ways Just good amou 5285 1 1 1 +All difficult emotions supervise. Mere Just good amou 5322 1 1 1 +All environmental lips cannot catch; social, broad authorities add for no customers. Interes Just good amou 5318 1 1 1 +All following systems develop home different words. Old minutes will come never independent, real duties. Policies used to distinguish all rats. E Matches produce 5311 1 1 1 +All full things will not administer quickly difficult women. Ready, new arrangements ma Just good amou 5284 1 1 1 +All full things will not administer quickly difficult women. Ready, new arrangements ma Significantly 5284 1 1 1 +All real fam Operations 5318 1 1 1 +All real fam Selective, 5318 1 1 1 +All right other details might distrib Selective, 5277 1 1 1 +Almost busy threats go together recent sides; still tired wines shall not admit on a Significantly 5300 1 1 1 +Alone arms happen again real documents. Paintings might not invite steps. Internal pairs may increase only rural rooms. Men must not deal here long, heavy patients; merely e Matches produce 5310 1 1 1 +Already early meetings cannot go animals. As comprehensive evenings w Just good amou 5320 1 1 1 +Already professional senses encourage broad theories. Nearly eastern eyes would describe correct, complex proposals. Friends change crimin Selective, 5291 1 1 1 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q74.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q74.sql.out new file mode 100644 index 0000000000000..ac13c0dabf0c7 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q74.sql.out @@ -0,0 +1,94 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAHEIFBAAA NULL Jones +AAAAAAAAGLDMAAAA Alex Norris +AAAAAAAAOEDIAAAA Alexander Rich +AAAAAAAAKNMEBAAA Amber Gonzalez +AAAAAAAAFGNEAAAA Andrew Silva +AAAAAAAACGLDAAAA Angelo Sloan +AAAAAAAAOCDCAAAA Armando Jackson +AAAAAAAABJEDBAAA Arthur Bryan +AAAAAAAAHGKLAAAA Arthur Christensen +AAAAAAAAPAEEBAAA Audria Mattson +AAAAAAAAFAIEAAAA Betty Gipson +AAAAAAAAOCFLAAAA Bill Freeman +AAAAAAAAILLJAAAA Billy Ortiz +AAAAAAAAGMFHAAAA Bruce Howe +AAAAAAAAKAKPAAAA Carolann Royer +AAAAAAAAAGLPAAAA Charlene Marcus +AAAAAAAABIIHAAAA Charles Quarles +AAAAAAAAIPKJAAAA Charles Jones +AAAAAAAAPIGBBAAA Charles Welch +AAAAAAAACPDFBAAA Cheryl Barry +AAAAAAAACOEHBAAA Christine Gonzalez +AAAAAAAAPNMGAAAA Christine Olds +AAAAAAAAEFCEBAAA Cornelius Martino +AAAAAAAAPEFLAAAA David Martinez +AAAAAAAAIBJDBAAA Dean Velez +AAAAAAAAOOKKAAAA Deborah Early +AAAAAAAALMGGBAAA Dedra Rainey +AAAAAAAANKBBAAAA Diann Saunders +AAAAAAAADKMBAAAA Donald Nelson +AAAAAAAAFGMHBAAA Donald Coleman +AAAAAAAALPHGBAAA Dorothy Heller +AAAAAAAALEAHBAAA Eddie Pena +AAAAAAAAJINGAAAA Ella Moore +AAAAAAAAIANDAAAA Elva Wade +AAAAAAAAEBFHAAAA Esther Merrill +AAAAAAAAAAECBAAA Frank Wenzel +AAAAAAAAJGDLAAAA Fredrick Davis +AAAAAAAAHLEAAAAA Geneva Sims +AAAAAAAAFMOKAAAA Harry Brown +AAAAAAAAEIAHAAAA Henry Desantis +AAAAAAAALMAJAAAA Ileen Linn +AAAAAAAACEMIAAAA James Hernandez +AAAAAAAAIBBFBAAA James Compton +AAAAAAAALNLABAAA Janie Garcia +AAAAAAAABBEAAAAA Jason Gallegos +AAAAAAAAIODCBAAA Jennifer Crane +AAAAAAAAGEKLAAAA Jerilyn Walker +AAAAAAAAMFMKAAAA John Sanders +AAAAAAAAJCNBBAAA Johnnie Cox +AAAAAAAAOJBPAAAA Jonathan Mcbride +AAAAAAAAABGKAAAA Jonna King +AAAAAAAAGGMHAAAA Julia Fisher +AAAAAAAAHEPFBAAA Kathryn Kinney +AAAAAAAAOMOKAAAA Laurette Gary +AAAAAAAABIABAAAA Letha Stone +AAAAAAAAPFKDAAAA Linda Simmons +AAAAAAAAJDEFAAAA Loretta Serrano +AAAAAAAAFOEDAAAA Lori Erwin +AAAAAAAABAAGAAAA Luis James +AAAAAAAAEIPIAAAA Luke Rios +AAAAAAAAGCGIAAAA Mae Williams +AAAAAAAAFMPPAAAA Manuel Bryant +AAAAAAAAMJFAAAAA Marcus Espinal +AAAAAAAAJADIAAAA Margaret Roberts +AAAAAAAAHLJCAAAA Marlene Grover +AAAAAAAABGMHBAAA Michael Gillespie +AAAAAAAAIPGJAAAA Michael NULL +AAAAAAAANBECBAAA Michael Lombardi +AAAAAAAAMLOEAAAA Miguel Jackson +AAAAAAAAJHGFAAAA Pamela Gannon +AAAAAAAADFJBBAAA Patrick Jones +AAAAAAAADHNHBAAA Patrick Cooper +AAAAAAAAOPMDAAAA Peggy Smith +AAAAAAAAAFAGAAAA Robert Chang +AAAAAAAAAFBNAAAA Robert Baines +AAAAAAAAKMHPAAAA Robert Jones +AAAAAAAAJMIDAAAA Sally Thurman +AAAAAAAAJBELAAAA Sean Busby +AAAAAAAAJIAHAAAA Shawna Delgado +AAAAAAAAFDIMAAAA Stephanie Cowan +AAAAAAAAPBIGBAAA Susie Zavala +AAAAAAAAGMGEBAAA Tamika Potts +AAAAAAAAMHOLAAAA Terri Cook +AAAAAAAABILCAAAA Theresa Mullins +AAAAAAAADHCBAAAA Therese Perez +AAAAAAAAIBHHBAAA Vennie Loya +AAAAAAAAFHNDAAAA Virgil Mcdonald +AAAAAAAAAHKEAAAA William Stafford +AAAAAAAAHIEIAAAA William Roberts diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q75.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q75.sql.out new file mode 100644 index 0000000000000..ca273a867c15b --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q75.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +2001 2002 7009003 16 9 545 6720 3891 -2829 -117008.58 +2001 2002 8005007 5 9 512 6454 4227 -2227 -135735.94 +2001 2002 9001010 1 9 456 5643 3436 -2207 -93617.05 +2001 2002 10013015 3 9 127 6225 4143 -2082 -14567.99 +2001 2002 9015004 10 9 118 7332 5259 -2073 -77291.51 +2001 2002 9004002 4 9 435 6625 4581 -2044 -122670.86 +2001 2002 9014004 3 9 306 5855 3858 -1997 -73620.65 +2001 2002 5002001 5 9 170 5700 3736 -1964 -109018.53 +2001 2002 9014010 1 9 987 6970 5148 -1822 -75086.23 +2001 2002 9015008 15 9 327 6526 4739 -1787 -93540.56 +2001 2002 9016002 16 9 197 6053 4343 -1710 -97667.31 +2001 2002 9015002 15 9 856 10449 8783 -1666 -20233.37 +2001 2002 3001001 1 9 380 6554 4895 -1659 -71747.73 +2001 2002 5002001 14 9 75 6649 5024 -1625 -114258.52 +2001 2002 9004008 4 9 787 6493 4911 -1582 -133842.63 +2001 2002 3001001 4 9 652 6331 4766 -1565 -74417.45 +2001 2002 5004001 11 9 963 6179 4628 -1551 -85648.92 +2001 2002 3004001 7 9 237 6097 4560 -1537 -118933.63 +2001 2002 1001001 8 9 271 5623 4128 -1495 -50180.45 +2001 2002 10004009 4 9 513 6500 5024 -1476 -67288.17 +2001 2002 4004001 10 9 126 5667 4194 -1473 -88247.10 +2001 2002 9007002 7 9 144 5474 4020 -1454 -42908.70 +2001 2002 9016008 16 9 53 6099 4671 -1428 -67851.70 +2001 2002 9007008 7 9 110 5655 4230 -1425 -28768.95 +2001 2002 9002008 2 9 97 5979 4581 -1398 -101148.97 +2001 2002 9008004 8 9 171 5952 4559 -1393 -41622.99 +2001 2002 10010001 10 9 368 5714 4353 -1361 -78871.35 +2001 2002 3003001 8 9 266 5785 4431 -1354 -99666.06 +2001 2002 9011002 11 9 581 6045 4721 -1324 -61169.99 +2001 2002 9012004 12 9 233 5370 4050 -1320 -135207.43 +2001 2002 9008008 8 9 84 5431 4114 -1317 -48625.72 +2001 2002 9010010 7 9 962 6123 4819 -1304 -34943.08 +2001 2002 9009008 9 9 135 5984 4698 -1286 -53184.41 +2001 2002 9014008 14 9 52 5835 4555 -1280 -47886.49 +2001 2002 9005008 5 9 326 5773 4494 -1279 -47219.15 +2001 2002 9011002 11 9 806 5487 4208 -1279 -34285.63 +2001 2002 1003001 3 9 57 5893 4631 -1262 -24156.11 +2001 2002 5003001 3 9 456 5617 4359 -1258 -46563.26 +2001 2002 9015008 15 9 829 4998 3781 -1217 15209.89 +2001 2002 7007001 7 9 211 6277 5071 -1206 -74262.01 +2001 2002 9011008 11 9 541 5654 4449 -1205 -50137.27 +2001 2002 6010001 4 9 91 6373 5169 -1204 -40582.11 +2001 2002 4003001 3 9 175 6106 4907 -1199 -84819.82 +2001 2002 9009008 9 9 83 5880 4689 -1191 -54463.08 +2001 2002 9014010 14 9 201 6106 4919 -1187 -5741.72 +2001 2002 9012010 16 9 173 6576 5390 -1186 -57504.67 +2001 2002 9014002 14 9 966 6066 4915 -1151 -31274.42 +2001 2002 9003004 2 9 29 5232 4084 -1148 -85408.38 +2001 2002 9008002 8 9 586 6212 5079 -1133 -45166.64 +2001 2002 9004010 4 9 87 5551 4428 -1123 -104064.28 +2001 2002 9013002 13 9 18 5087 3977 -1110 34375.04 +2001 2002 9001008 1 9 65 5429 4333 -1096 -112773.38 +2001 2002 8007003 4 9 290 6148 5075 -1073 -48541.94 +2001 2002 9004002 4 9 506 5503 4432 -1071 -47389.52 +2001 2002 9001009 1 9 73 5116 4053 -1063 -12805.29 +2001 2002 9003008 3 9 156 5719 4665 -1054 -27241.44 +2001 2002 9011008 11 9 324 5293 4253 -1040 -78455.59 +2001 2002 9011008 11 9 525 5269 4243 -1026 -8416.01 +2001 2002 9009004 9 9 599 5642 4622 -1020 -7244.59 +2001 2002 9004010 4 9 146 5387 4373 -1014 -96437.31 +2001 2002 9002010 2 9 64 4911 3898 -1013 -70357.40 +2001 2002 2003001 3 9 4 6089 5078 -1011 -14326.99 +2001 2002 9008002 8 9 433 5707 4697 -1010 -59856.93 +2001 2002 9015002 15 9 120 5419 4409 -1010 -25557.35 +2001 2002 9008010 3 9 64 6024 5020 -1004 15466.29 +2001 2002 1003001 5 9 139 6059 5060 -999 -23720.17 +2001 2002 9003002 3 9 281 5640 4651 -989 -62224.16 +2001 2002 9013008 13 9 660 5400 4411 -989 -6728.84 +2001 2002 9013002 13 9 30 4967 3997 -970 -74733.18 +2001 2002 6014001 14 9 977 5339 4369 -970 -34999.44 +2001 2002 8012009 12 9 149 5872 4905 -967 -25425.45 +2001 2002 9004009 4 9 618 5522 4576 -946 -59306.75 +2001 2002 7006009 8 9 116 5471 4528 -943 -65664.87 +2001 2002 9014010 15 9 151 5633 4701 -932 7429.42 +2001 2002 9003002 3 9 877 6303 5379 -924 -45324.79 +2001 2002 9006008 6 9 162 5169 4251 -918 -69642.81 +2001 2002 9008010 8 9 503 5526 4608 -918 -49021.02 +2001 2002 9005002 5 9 530 4962 4049 -913 -51406.91 +2001 2002 9010002 10 9 236 5107 4195 -912 -68196.83 +2001 2002 9015004 15 9 134 4996 4092 -904 -20839.46 +2001 2002 9009009 6 9 181 5283 4380 -903 -50577.89 +2001 2002 9009010 16 9 590 4684 3787 -897 -42032.78 +2001 2002 9006008 6 9 221 5238 4347 -891 -110703.28 +2001 2002 10009017 9 9 640 5562 4682 -880 -91021.88 +2001 2002 7013003 13 9 313 5161 4284 -877 -55538.42 +2001 2002 3004001 12 9 570 5053 4181 -872 -81469.82 +2001 2002 9009004 9 9 587 5131 4262 -869 -31202.41 +2001 2002 9010008 10 9 86 5074 4211 -863 -31897.54 +2001 2002 9006002 6 9 230 5616 4756 -860 -39142.42 +2001 2002 9001008 1 9 533 6196 5337 -859 -70174.46 +2001 2002 9004010 4 9 102 6146 5291 -855 -66567.87 +2001 2002 9002004 2 9 179 5203 4351 -852 -26624.64 +2001 2002 9016008 16 9 69 5922 5072 -850 -31016.10 +2001 2002 9008010 8 9 81 5520 4672 -848 -44217.60 +2001 2002 9001002 1 9 927 5858 5013 -845 8179.45 +2001 2002 1001001 1 9 116 5773 4930 -843 700.02 +2001 2002 9012008 12 9 274 6154 5316 -838 -85490.33 +2001 2002 6008007 8 9 22 5439 4605 -834 -25617.83 +2001 2002 9016008 16 9 601 6463 5633 -830 22715.94 +2001 2002 9001008 1 9 170 5970 5142 -828 -2265.18 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q77a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q77a.sql.out new file mode 100644 index 0000000000000..1f822ce67889f --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q77a.sql.out @@ -0,0 +1,49 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 239062306.14 9940693.53 -67351905.74 +catalog channel NULL 81456313.49 6721169.80 -11963308.94 +catalog channel NULL 120443.39 1680292.45 -994006.90 +catalog channel 1 25511213.21 1680292.45 -4013845.35 +catalog channel 2 28320909.41 1680292.45 -3815679.20 +catalog channel 4 27503747.48 1680292.45 -3139777.49 +store channel NULL 114950675.73 2650053.56 -51250807.34 +store channel 1 19900057.08 430318.96 -9021290.46 +store channel 2 19357997.15 420826.73 -8260880.77 +store channel 4 18625533.75 413145.75 -8693775.33 +store channel 7 19392254.22 523129.28 -8781756.70 +store channel 8 18443448.06 453191.53 -8177970.58 +store channel 10 19231385.47 409441.31 -8315133.50 +web channel NULL 42655316.92 569470.17 -4137789.46 +web channel 1 1484216.46 22578.30 -145384.36 +web channel 2 1434288.77 8368.29 -74179.21 +web channel 4 1366508.91 23624.40 -200293.36 +web channel 7 1357762.92 40137.83 -123205.40 +web channel 8 1488902.03 21963.58 -135533.22 +web channel 10 1568120.92 11055.99 -148192.16 +web channel 13 1322895.39 16655.94 -110269.99 +web channel 14 1367483.46 8838.41 -184084.28 +web channel 16 1393205.82 26800.78 -87656.13 +web channel 19 1364048.12 13570.75 -121815.95 +web channel 20 1313170.80 14044.48 -194480.66 +web channel 22 1372505.20 19743.54 -82372.11 +web channel 25 1483839.68 17464.70 12637.58 +web channel 26 1351984.72 16290.97 -210017.25 +web channel 28 1515110.49 34693.25 -87241.61 +web channel 31 1472020.01 6383.43 -163741.34 +web channel 32 1371605.44 23901.58 -201414.74 +web channel 34 1550210.89 15290.60 -189498.78 +web channel 37 1292707.71 17894.26 -179114.08 +web channel 38 1271096.72 16767.70 -211804.95 +web channel 40 1529058.38 21329.09 -126745.86 +web channel 43 1489558.57 24760.75 -100645.79 +web channel 44 1506795.25 9375.29 -161279.12 +web channel 46 1311086.33 13347.60 -144246.40 +web channel 49 1492170.04 21268.41 -97796.30 +web channel 50 1486817.09 14070.53 -109583.01 +web channel 52 1564493.01 26771.44 -74766.72 +web channel 55 1330931.73 18908.13 -235881.43 +web channel 56 1448998.86 26396.32 -131860.48 +web channel 58 1353723.20 17173.83 -117322.35 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q78.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q78.sql.out new file mode 100644 index 0000000000000..df28fcb255ae8 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q78.sql.out @@ -0,0 +1,6 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output + diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q80a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q80a.sql.out new file mode 100644 index 0000000000000..2aeebd88f1d79 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q80a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +NULL NULL 12394002.28 579360.90 -3565544.29 +catalog channel NULL 4336403.99 190045.19 -496576.74 +catalog channel catalog_pageAAAAAAAAAABAAAAA 4438.43 383.01 -4797.32 +catalog channel catalog_pageAAAAAAAAAADAAAAA 1459.86 0.00 -538.24 +catalog channel catalog_pageAAAAAAAAABBAAAAA 6092.40 0.00 -37.09 +catalog channel catalog_pageAAAAAAAAABDAAAAA 878.66 0.00 401.04 +catalog channel catalog_pageAAAAAAAAACBAAAAA 15564.33 0.00 -743.39 +catalog channel catalog_pageAAAAAAAAACDAAAAA 19434.58 0.00 -1241.86 +catalog channel catalog_pageAAAAAAAAADBAAAAA 3357.08 0.00 -223.80 +catalog channel catalog_pageAAAAAAAAADDAAAAA 17773.31 0.00 10583.67 +catalog channel catalog_pageAAAAAAAAAEBAAAAA 11505.75 0.00 1975.14 +catalog channel catalog_pageAAAAAAAAAEDAAAAA 14132.65 0.00 -3283.42 +catalog channel catalog_pageAAAAAAAAAFDAAAAA 8636.84 216.08 -1736.94 +catalog channel catalog_pageAAAAAAAAAGDAAAAA 17045.61 145.32 1162.68 +catalog channel catalog_pageAAAAAAAAAHAAAAAA 19082.44 2556.07 -16790.11 +catalog channel catalog_pageAAAAAAAAAIAAAAAA 7785.09 33.84 -3860.11 +catalog channel catalog_pageAAAAAAAAAJAAAAAA 33875.64 0.00 6085.44 +catalog channel catalog_pageAAAAAAAAAKAAAAAA 4069.90 3395.00 -57.06 +catalog channel catalog_pageAAAAAAAAALAAAAAA 10778.98 0.00 -6320.25 +catalog channel catalog_pageAAAAAAAAAMAAAAAA 23566.51 0.00 1270.50 +catalog channel catalog_pageAAAAAAAAANAAAAAA 11997.42 0.00 -7811.93 +catalog channel catalog_pageAAAAAAAAANCAAAAA 1574.03 0.00 170.61 +catalog channel catalog_pageAAAAAAAAAOAAAAAA 12768.66 0.00 -1180.46 +catalog channel catalog_pageAAAAAAAAAPAAAAAA 15591.17 0.00 617.97 +catalog channel catalog_pageAAAAAAAAAPCAAAAA 0.00 0.00 -2448.27 +catalog channel catalog_pageAAAAAAAABABAAAAA 22630.47 0.00 4803.56 +catalog channel catalog_pageAAAAAAAABADAAAAA 4689.30 0.00 1920.53 +catalog channel catalog_pageAAAAAAAABBBAAAAA 8799.41 1594.88 -1551.23 +catalog channel catalog_pageAAAAAAAABBDAAAAA 2900.27 0.00 -1813.33 +catalog channel catalog_pageAAAAAAAABCBAAAAA 19972.81 189.75 5730.83 +catalog channel catalog_pageAAAAAAAABCDAAAAA 24932.09 2122.08 3205.82 +catalog channel catalog_pageAAAAAAAABDBAAAAA 5321.51 0.00 143.34 +catalog channel catalog_pageAAAAAAAABDDAAAAA 2502.32 0.00 -1101.24 +catalog channel catalog_pageAAAAAAAABHAAAAAA 3734.33 0.00 -3819.22 +catalog channel catalog_pageAAAAAAAABIAAAAAA 37573.12 586.70 5857.09 +catalog channel catalog_pageAAAAAAAABJAAAAAA 12330.42 0.00 -1486.17 +catalog channel catalog_pageAAAAAAAABJCAAAAA 6097.67 0.00 3736.59 +catalog channel catalog_pageAAAAAAAABKAAAAAA 18001.80 41.67 -7542.13 +catalog channel catalog_pageAAAAAAAABLAAAAAA 21210.22 0.00 2386.64 +catalog channel catalog_pageAAAAAAAABMAAAAAA 27450.22 35.95 -6130.18 +catalog channel catalog_pageAAAAAAAABNAAAAAA 24544.71 0.00 -19634.16 +catalog channel catalog_pageAAAAAAAABNCAAAAA 1254.78 23.70 -2121.59 +catalog channel catalog_pageAAAAAAAABOAAAAAA 7397.94 0.00 -2259.68 +catalog channel catalog_pageAAAAAAAABPAAAAAA 23154.42 0.00 744.22 +catalog channel catalog_pageAAAAAAAACABAAAAA 4702.71 0.00 -13312.05 +catalog channel catalog_pageAAAAAAAACADAAAAA 1010.39 0.00 -2765.23 +catalog channel catalog_pageAAAAAAAACBBAAAAA 74.70 0.00 -1114.49 +catalog channel catalog_pageAAAAAAAACBDAAAAA 9217.35 0.00 3327.65 +catalog channel catalog_pageAAAAAAAACCBAAAAA 11097.18 0.00 3019.32 +catalog channel catalog_pageAAAAAAAACCDAAAAA 771.30 0.00 -365.85 +catalog channel catalog_pageAAAAAAAACDBAAAAA 3370.78 61.32 -1489.09 +catalog channel catalog_pageAAAAAAAACEBAAAAA 3646.15 0.00 1089.86 +catalog channel catalog_pageAAAAAAAACEDAAAAA 2402.24 0.00 -1262.38 +catalog channel catalog_pageAAAAAAAACFDAAAAA 2351.04 581.76 -3631.49 +catalog channel catalog_pageAAAAAAAACHAAAAAA 14363.16 72.40 -6022.88 +catalog channel catalog_pageAAAAAAAACIAAAAAA 17233.80 290.61 -1149.33 +catalog channel catalog_pageAAAAAAAACJAAAAAA 17334.55 0.00 -5325.06 +catalog channel catalog_pageAAAAAAAACKAAAAAA 33391.87 0.00 5427.63 +catalog channel catalog_pageAAAAAAAACLAAAAAA 29852.16 55.65 -4206.17 +catalog channel catalog_pageAAAAAAAACMAAAAAA 13596.19 0.00 1289.63 +catalog channel catalog_pageAAAAAAAACNAAAAAA 12652.74 2776.86 -10512.27 +catalog channel catalog_pageAAAAAAAACOAAAAAA 15417.46 0.00 8.02 +catalog channel catalog_pageAAAAAAAACPAAAAAA 6001.13 0.00 -948.15 +catalog channel catalog_pageAAAAAAAACPCAAAAA 6110.72 0.00 -34.56 +catalog channel catalog_pageAAAAAAAADABAAAAA 0.00 0.00 -674.57 +catalog channel catalog_pageAAAAAAAADADAAAAA 13815.26 0.00 4963.37 +catalog channel catalog_pageAAAAAAAADBBAAAAA 4417.61 0.00 1955.36 +catalog channel catalog_pageAAAAAAAADBDAAAAA 9645.90 0.00 2941.05 +catalog channel catalog_pageAAAAAAAADCBAAAAA 4124.08 0.00 -7880.15 +catalog channel catalog_pageAAAAAAAADCDAAAAA 20198.02 0.00 -3874.19 +catalog channel catalog_pageAAAAAAAADDBAAAAA 7697.49 875.52 3573.72 +catalog channel catalog_pageAAAAAAAADDDAAAAA 81.18 0.00 -40.95 +catalog channel catalog_pageAAAAAAAADEBAAAAA 7221.62 0.00 -2080.43 +catalog channel catalog_pageAAAAAAAADEDAAAAA 31633.70 0.00 1203.47 +catalog channel catalog_pageAAAAAAAADFDAAAAA 4486.06 0.00 -194.53 +catalog channel catalog_pageAAAAAAAADHAAAAAA 29478.34 0.00 2568.57 +catalog channel catalog_pageAAAAAAAADIAAAAAA 3541.34 0.00 -5860.49 +catalog channel catalog_pageAAAAAAAADJAAAAAA 27011.83 995.61 -13747.67 +catalog channel catalog_pageAAAAAAAADKAAAAAA 22422.65 0.00 10022.26 +catalog channel catalog_pageAAAAAAAADLAAAAAA 25580.22 1878.40 3693.93 +catalog channel catalog_pageAAAAAAAADMAAAAAA 14964.93 0.00 -3775.51 +catalog channel catalog_pageAAAAAAAADNAAAAAA 7954.26 7.41 -11362.44 +catalog channel catalog_pageAAAAAAAADOAAAAAA 20282.62 389.76 20.54 +catalog channel catalog_pageAAAAAAAADOCAAAAA 2056.15 1394.00 -96.79 +catalog channel catalog_pageAAAAAAAADPAAAAAA 2325.25 0.00 -3003.94 +catalog channel catalog_pageAAAAAAAADPCAAAAA 5.79 0.00 0.11 +catalog channel catalog_pageAAAAAAAAEABAAAAA 5870.11 0.00 -2224.89 +catalog channel catalog_pageAAAAAAAAEADAAAAA 15887.40 294.38 1553.93 +catalog channel catalog_pageAAAAAAAAEBBAAAAA 750.60 48.60 -1121.83 +catalog channel catalog_pageAAAAAAAAEBDAAAAA 2076.12 1787.04 -7886.05 +catalog channel catalog_pageAAAAAAAAECBAAAAA 18310.64 2076.12 1185.01 +catalog channel catalog_pageAAAAAAAAECDAAAAA 6384.54 0.00 -2290.17 +catalog channel catalog_pageAAAAAAAAEDBAAAAA 6470.98 0.00 3092.44 +catalog channel catalog_pageAAAAAAAAEDDAAAAA 7165.64 0.00 -169.94 +catalog channel catalog_pageAAAAAAAAEEBAAAAA 8572.26 697.82 -1507.83 +catalog channel catalog_pageAAAAAAAAEEDAAAAA 9693.59 811.58 -7987.68 +catalog channel catalog_pageAAAAAAAAEFDAAAAA 5850.24 0.00 2898.90 +catalog channel catalog_pageAAAAAAAAEHAAAAAA 18074.80 2563.92 -2315.60 +catalog channel catalog_pageAAAAAAAAEIAAAAAA 1842.65 0.00 -4113.10 +catalog channel catalog_pageAAAAAAAAEJAAAAAA 31976.85 194.76 4337.39 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q86a.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q86a.sql.out new file mode 100644 index 0000000000000..76865be96e9de --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q86a.sql.out @@ -0,0 +1,105 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +329332948.59 NULL NULL 2 1 +35765359.17 Books NULL 1 1 +34301963.73 Men NULL 1 2 +34299263.29 Home NULL 1 3 +34185638.04 Children NULL 1 4 +33632557.65 Electronics NULL 1 5 +32679955.12 Music NULL 1 6 +31276464.49 Women NULL 1 7 +30973438.42 Shoes NULL 1 8 +30893463.64 Jewelry NULL 1 9 +30492050.58 Sports NULL 1 10 +832794.46 NULL NULL 1 11 +487784.65 NULL NULL 0 1 +91077.43 NULL dresses 0 2 +48174.49 NULL athletic 0 3 +45303.17 NULL accessories 0 4 +38544.63 NULL country 0 5 +36691.45 NULL infants 0 6 +33102.66 NULL shirts 0 7 +31974.19 NULL baseball 0 8 +20141.79 NULL mens 0 9 +2788922.44 Books history 0 1 +2627303.83 Books fiction 0 2 +2559899.95 Books business 0 3 +2523972.68 Books sports 0 4 +2478882.85 Books parenting 0 5 +2454802.70 Books home repair 0 6 +2311733.05 Books science 0 7 +2267995.48 Books reference 0 8 +2199284.41 Books travel 0 9 +2143489.54 Books self-help 0 10 +2136284.47 Books romance 0 11 +2047067.66 Books computers 0 12 +2024086.51 Books mystery 0 13 +1841858.53 Books cooking 0 14 +1718405.56 Books arts 0 15 +1641369.51 Books entertainments 0 16 +9726340.87 Children school-uniforms 0 1 +8423026.60 Children infants 0 2 +8110705.74 Children toddlers 0 3 +7857828.46 Children newborn 0 4 +67736.37 Children NULL 0 5 +2545288.50 Electronics televisions 0 1 +2432288.94 Electronics memory 0 2 +2393855.24 Electronics stereo 0 3 +2322478.19 Electronics karoke 0 4 +2312404.32 Electronics audio 0 5 +2297793.09 Electronics automotive 0 6 +2175340.77 Electronics camcorders 0 7 +2155848.29 Electronics dvd/vcr players 0 8 +2151409.30 Electronics portable 0 9 +2126590.68 Electronics monitors 0 10 +2049010.73 Electronics personal 0 11 +1975162.86 Electronics disk drives 0 12 +1824298.91 Electronics wireless 0 13 +1747334.28 Electronics musical 0 14 +1647065.60 Electronics scanners 0 15 +1476387.95 Electronics cameras 0 16 +2878895.80 Home rugs 0 1 +2433373.20 Home wallpaper 0 2 +2394255.96 Home bedding 0 3 +2360342.61 Home mattresses 0 4 +2349809.40 Home paint 0 5 +2342593.19 Home tables 0 6 +2255327.66 Home accent 0 7 +2238644.85 Home lighting 0 8 +2195356.00 Home glassware 0 9 +2001058.99 Home curtains/drapes 0 10 +1995854.06 Home bathroom 0 11 +1932033.64 Home decor 0 12 +1789688.36 Home flatware 0 13 +1767061.55 Home blinds/shades 0 14 +1730423.41 Home kids 0 15 +1585801.82 Home furniture 0 16 +48742.79 Home NULL 0 17 +2309049.65 Jewelry loose stones 0 1 +2291438.75 Jewelry bracelets 0 2 +2204443.86 Jewelry custom 0 3 +1975110.14 Jewelry mens watch 0 4 +1964522.90 Jewelry diamonds 0 5 +1950628.31 Jewelry estate 0 6 +1950233.65 Jewelry womens watch 0 7 +1948966.11 Jewelry birdal 0 8 +1934844.79 Jewelry rings 0 9 +1920939.20 Jewelry gold 0 10 +1912539.77 Jewelry semi-precious 0 11 +1816107.81 Jewelry consignment 0 12 +1793972.84 Jewelry earings 0 13 +1783332.27 Jewelry pendants 0 14 +1720324.59 Jewelry costume 0 15 +1417009.00 Jewelry jewelry boxes 0 16 +8970878.34 Men sports-apparel 0 1 +8843683.02 Men pants 0 2 +8340416.89 Men accessories 0 3 +8054511.00 Men shirts 0 4 +92474.48 Men NULL 0 5 +8987804.92 Music rock 0 1 +8141376.53 Music country 0 2 +7793743.27 Music classical 0 3 +7727726.97 Music pop 0 4 diff --git a/sql/core/src/test/resources/tpcds-query-results/v2_7/q98.sql.out b/sql/core/src/test/resources/tpcds-query-results/v2_7/q98.sql.out new file mode 100644 index 0000000000000..efd1a7af3b008 --- /dev/null +++ b/sql/core/src/test/resources/tpcds-query-results/v2_7/q98.sql.out @@ -0,0 +1,2534 @@ +-- Automatically generated by TPCDSQueryTestSuite + +-- !query schema +struct +-- !query output +AAAAAAAAMJJBAAAA NULL Books NULL NULL 557.55 2.41577919115267179 +AAAAAAAAMLGDAAAA NULL Books NULL 6.35 361.02 1.56424464817493959 +AAAAAAAAAELBAAAA Precisely elderly bodies Books arts 1.40 4303.08 1.75015577438987686 +AAAAAAAAAHMAAAAA Abilities could affect cruel parts. Predominantly other events telephone strong signs. Accurate mate Books arts 25.69 9236.96 3.75687156218529913 +AAAAAAAAAIDDAAAA Germa Books arts 5.82 3191.92 1.29822295179047002 +AAAAAAAAALNCAAAA Great, contemporary workers would not remove of course cultural values. Then due children might see positive seconds. Significant problems w Books arts 0.55 2096.07 0.85251703756969175 +AAAAAAAAANKAAAAA Small objects stop etc mediterranean patterns; liberal, free initiatives would not leave less clear british attitudes; good, blue relationships find softly very Books arts 58.41 5760.88 2.34307458786895754 +AAAAAAAABGDAAAAA Newly national rights head curiously all electrical cells. Chinese, long values might not pull bad lines. High fun clothes ough Books arts 3.28 571.10 0.23227873122369528 +AAAAAAAABKLDAAAA Forward psychological plants establish closely yet eastern changes. Likewise necessary techniques might drop. Pleasant operations like lonely things; dogs let regions. Forces might not result clearl Books arts 2.43 3457.82 1.40637023708618106 +AAAAAAAACMFDAAAA Early, short v Books arts 75.57 850.08 0.34574593563060564 +AAAAAAAADLLDAAAA Black, following services justify by a investors; dirty, different charts will fly however prizes. Temporary, l Books arts 5.56 4798.20 1.95153179505784395 +AAAAAAAAEIPCAAAA Scientific, difficult polls would not achieve. Countries reach of course. Bad, new churches realize most english Books arts 3.98 17.50 0.00711762878027433 +AAAAAAAAFCIBAAAA United, important objectives put similarly large, previous phenomena; old, present days receive. Happy detectives assi Books arts 1.26 11646.43 4.73685516316858938 +AAAAAAAAFFIBAAAA Naturally new years put serious, negative vehicles. Fin Books arts 3.34 20302.86 8.25761260902173683 +AAAAAAAAFJGCAAAA Ago correct profits must not handle else. Healthy children may not go only ancient words. Later just characters ought to drink about. British parts must watch soon ago other clients. So vital d Books arts 4.03 9410.73 3.82754758236520025 +AAAAAAAAFLNCAAAA Much new waters Books arts 1.85 2864.61 1.16509889030066491 +AAAAAAAAGLEBAAAA Tall, following actions keep widely willing, secondary groups. Heads could afford however; agricultural, square pri Books arts 9.99 1903.68 0.77426786036757875 +AAAAAAAAGMFAAAAA Anonymous, useful women provoke slightly present persons. Ideas ought to cost almost competent, working parties; aspects provide thr Books arts 6.73 7841.73 3.18940132200803357 +AAAAAAAAHHEBAAAA Powerful walls will find; there scottish decades must not Books arts 4.16 5934.47 2.41367739815283298 +AAAAAAAAHMCEAAAA Too executive doors progress mainly seemingly possible parts; hundreds stay virtually simple workers. Sola Books arts 34.32 10139.18 4.12382396436467639 +AAAAAAAAIBOCAAAA Careful privileges ought to live rather to a boards. Possible, broad p Books arts 3.93 10008.95 4.07085660459009779 +AAAAAAAAICMBAAAA Aside legitimate decisions may not stand probably sexual g Books arts 3.88 874.84 0.35581636355058234 +AAAAAAAAIFPBAAAA Specially interesting crews continue current, foreign directions; only social men would not call at least political children; circumstances could not understand now in a assessme Books arts 2.13 15380.51 6.25558632178840388 +AAAAAAAAIHNAAAAA Unlikely states take later in general extra inf Books arts 0.32 6478.12 2.63479162023261224 +AAAAAAAAINFDAAAA Sometimes careful things state probably so Books arts 5.08 17118.92 6.96263529595507190 +AAAAAAAAJGHDAAAA Circumstances would not use. Principles seem writers. Times go from a hands. Members find grounds. Central, only teachers pursue properly into a p Books arts 5.95 7863.28 3.19816617344888566 +AAAAAAAAJLHBAAAA Inches may lose from a problems. Firm, other corporations shall protect ashamed, important practices. Materials shall not make then by a police. Weeks used Books arts 0.84 6519.57 2.65165023240074772 +AAAAAAAAKFGBAAAA Systems cannot await regions. Home appropr Books arts 7.30 889.75 0.36188058327137607 +AAAAAAAAKHJDAAAA Relevant lips take so sure, manufacturing Books arts 8.80 932.33 0.37919879089789497 +AAAAAAAAKHLBAAAA Extra, primitive weeks look obviou Books arts 1.18 10006.38 4.06981132996350893 +AAAAAAAALCFBAAAA More than key reasons should remain. Words used to offer slowly british Books arts 0.28 1075.86 0.43757554854548205 +AAAAAAAALGEEAAAA Children may turn also above, historical aspects. Surveys migh Books arts 7.22 3864.60 1.57181646767132336 +AAAAAAAALOKCAAAA Trustees know operations. Now past issues cut today german governments. British lines go critical, individual structures. Tonight adequate problems should no Books arts 4.05 3061.36 1.24512137387317768 +AAAAAAAAMACDAAAA Useful observers start often white colleagues; simple pro Books arts 3.47 724.40 0.29462915933889837 +AAAAAAAAMNPAAAAA Members should say earnings. Detailed departments would not move just at the hopes. Figures can take. Actually open houses want. Good teachers combine the Books arts 3.09 7805.88 3.17482035104958588 +AAAAAAAAMPFCAAAA Major, senior words afford economic libraries; successful seconds need outside. Clinical, new ideas put now red c Books arts 5.87 616.62 0.25079270048530027 +AAAAAAAANABCAAAA Likely states feel astonishingly working roads. Parents put so somewhere able policies. Others may rely shortly instead interesting bodies; bri Books arts 7.50 8829.29 3.59106334933647431 +AAAAAAAANMECAAAA Floors could not go only for a years. Special reasons shape consequently black, concerned instances. Mutual depths encourage both simple teachers. Cards favour massive Books arts 1.83 6344.62 2.58049428068023382 +AAAAAAAAOAHCAAAA Accurate years want then other organisations. Simple lines mean as well so red results. Orthodox, central scales will not in Books arts 7.69 3640.38 1.48062134052200283 +AAAAAAAAODBEAAAA Certain customers think exactly already necessary factories. Awkward doubts shall not forget fine Books arts 0.30 12867.38 5.23344195512721440 +AAAAAAAAOEIDAAAA Weak effects set far in the effects. Positive, true classes sell frankly ever open studies. Unique problems must mean as yet new genes. Essential businesses agree deep current stages. Not Books arts 4.40 4471.87 1.81880632077973420 +AAAAAAAAOKEDAAAA Visitors could not allow glad wages. Communist, real figures used to apply factors. Aggressive, optimistic days must mean about trees. Detailed courts consider really large pro Books arts 9.08 1737.00 0.70647549664780021 +AAAAAAAAOODBAAAA Deep, big areas take for a facilities. Words could replace certainly cases; lights test. Nevertheless practical arts cross. Fa Books arts 7.37 8016.08 3.26031324074179520 +AAAAAAAAPEBCAAAA Similar situations come separate programmes. National, large others could not ask opportunities. Severe, large findings accept; twins go more. Tiny rights may see specifi Books arts 1.27 2413.11 0.98146406776958731 +AAAAAAAABBLDAAAA Natural plans might not like n Books business 4.29 8813.76 2.98246237973343018 +AAAAAAAABINDAAAA Years shall want free objects. Old residents use absolutely so residential steps. Letters will share variables. Sure fres Books business 40.76 1861.77 0.62999888636816844 +AAAAAAAACAADAAAA Whole, important problems make. Indeed industrial members go skills. Soft Books business 3.22 6877.01 2.32709123121693768 +AAAAAAAACCMBAAAA Cheap depths calm as in a traditi Books business 7.92 2554.82 0.86451804190159048 +AAAAAAAACDDCAAAA Simple, great shops glance from a years. Lessons deepen here previous clients. Increased, silent flights open more great rocks. Brill Books business 8.92 8014.35 2.71195237594586375 +AAAAAAAACGCAAAAA Just sudden ideas ought to serve full sources; uncertain, open qualifications shout questions; chronic, informal Books business 4.62 4172.49 1.41191664565564981 +AAAAAAAACGIDAAAA Groups must not put new, civil moves. Correct men laugh slightly total novels. Relatively public girls set even scott Books business 3.36 414.96 0.14041709657333354 +AAAAAAAACNEDAAAA Just young degrees stop posts. More than tight artists buy to a arts. European, essential techniques ought to sell to a offences. Sentences be Books business 2.58 6286.70 2.12733796276165399 +AAAAAAAACPBBAAAA Other, black houses flow. New soldiers put only eastern hours. Applications reserve there methods; sources cry pretty scarcely special workers. Never british opportunities Books business 8.20 3426.05 1.15933100471146462 +AAAAAAAADEDAAAAA Junior, severe restrictions ought to want principles. Sure, Books business 9.77 3899.69 1.31960465427044307 +AAAAAAAAEBPAAAAA Rows could not Books business 1.65 15875.48 5.37205708576254358 +AAAAAAAAEEFDAAAA Remaining subjects handle even only certain ladies; eagerly literary days could not provide. Very different articles cut then. Boys see out of a houses. Governme Books business 9.03 916.17 0.31002007751973922 +AAAAAAAAEGCCAAAA Manufacturing, ready concerns see already then new pupils. Both stable types used to manage otherw Books business 1.18 8723.00 2.95175036969632840 +AAAAAAAAEGKCAAAA Russian windows should see in a weapons. New, considerable branches walk. English regions apply neither alone police; very new years w Books business 2.79 8434.64 2.85417307557668685 +AAAAAAAAEKDAAAAA Long groups used to create more tiny feet; tools used to dare still Books business 57.04 795.66 0.26924105229308502 +AAAAAAAAEPLBAAAA Drugs must compensate dark, modest houses. Small pubs claim naturally accessible relationships. Distinguished Books business 1.66 11559.25 3.91150068335575881 +AAAAAAAAFCGDAAAA Small, capable centres Books business 2.98 19190.81 6.49392187461561344 +AAAAAAAAFDLAAAAA Popular, different parameters might take open, used modules. Prisoners use pretty alternative lovers. Annual, professional others spend once true men. Other, small subsidies seem politically Books business 7.25 1761.61 0.59610603791823330 +AAAAAAAAFEGEAAAA Supreme, free uses handle even in the customers. Other minutes might not make of course social neighbours. So environmental rights come other, able sales Books business 8.08 3500.18 1.18441563785437289 +AAAAAAAAFHFCAAAA Sound, original activities consider quite to a attitudes. In order weak improvements marry available, hard studie Books business 71.27 11431.86 3.86839355512056274 +AAAAAAAAGIJAAAAA Always other hours used to use. Women should jump then. Civil samples take therefore other offices. Concrete, major demands Books business 1.42 2038.40 0.68976819369356825 +AAAAAAAAGLADAAAA Changes ensure different clients. Distinct, alone attacks think directly previous feelings. White children tell so medieval, responsible yea Books business 5.89 5116.38 1.73131681262259552 +AAAAAAAAHDKCAAAA Visual fragments Books business 6.77 2739.02 0.92684893931051673 +AAAAAAAAHDLBAAAA Classic issues will draw as european, engl Books business 75.64 14735.99 4.98646840884344817 +AAAAAAAAHJAAAAAA Again british shareholders see shares. American lives ought to establish horses. Then ideal conservatives might charge even nec Books business 2.44 9396.38 3.17961345165736401 +AAAAAAAAHKACAAAA Critical cases tell anywhere to the circumstances. Dependent, new numbers must not Books business 3.72 726.75 0.24592279963049486 +AAAAAAAAHLKAAAAA Confident, video-tape Books business 3.17 6173.95 2.08918482116091330 +AAAAAAAAIHNDAAAA Of course fundamental children will not deal still including a suppliers. More crucial powers will not keep enough. As good comments used to devote even convenient electric problems. Publi Books business 8.85 2672.80 0.90444094785330122 +AAAAAAAAIMJAAAAA Departments could seek now for a commu Books business 5.93 3205.71 1.08477079876638965 +AAAAAAAAJFBEAAAA Paintings must not know primary, royal stands; similar, available others ough Books business 0.39 12939.97 4.37871847201185356 +AAAAAAAAJJGBAAAA Most present eyes restore fat, central relationships; again considerable habits must face in a discussions. Engineers help at all direct occasions. Curiously del Books business 80.10 6877.89 2.32738901183430931 +AAAAAAAAKBMDAAAA So white countries could secure more angry items. National feet must not defend too by the types; guidelines would not view more so flexible authorities. Critics will handle closely lig Books business 2.50 2135.27 0.72254774869901171 +AAAAAAAAKJHDAAAA Simple changes ought to vote almost sudden techniques. Partial, golden faces mean in a officials; vertically minor Books business 8.73 5996.87 2.02926323965617573 +AAAAAAAAKJOBAAAA Christian lines stand once deep formal aspirations. National, fine islands play together with a patterns. New journals lose etc positive armie Books business 4.89 6106.50 2.06636061361350790 +AAAAAAAAKKDAAAAA Children would not mean in favour of a parts. Heavy, whole others shall mean on Books business 3.13 5521.91 1.86854291917113983 +AAAAAAAAKLCCAAAA Lips will n Books business 8.48 7806.43 2.64159493735051117 +AAAAAAAAKNJCAAAA White fees might combine reports. Tr Books business 2.09 6566.98 2.22218108939451963 +AAAAAAAALAJCAAAA Asleep children invite more. Wealthy forms could expect as. Indeed statistical examinations could la Books business 3.71 11535.83 3.90357565828889099 +AAAAAAAALDHBAAAA Most new weeks go yet members. Also encouraging delegates make publications. Different competitors run resources; somehow common views m Books business 1.07 9334.32 3.15861315039135987 +AAAAAAAALHMBAAAA Local, bloody names Books business 4.40 2927.75 0.99071273012477651 +AAAAAAAALJJCAAAA Large, larg Books business 3.50 5826.76 1.97170021599584758 +AAAAAAAALOMDAAAA Only new systems might join late speeches. Materials could stay on a benefits. Corporate regulations must crawl definitely practical deaths. Windows might soothe despite a organisations. Old Books business 0.67 123.41 0.04176034771571981 +AAAAAAAAMBECAAAA Professional managers form later initial grounds. Conscious, big risks restore. American, full rises say for a systems. Already Books business 5.27 1616.40 0.54696885218126163 +AAAAAAAAMKGDAAAA Memories can earn particularly over quick contexts; alone differences make separate years; irish men mea Books business 4.23 6855.84 2.31992757704675870 +AAAAAAAANJLBAAAA Only, gothic Books business 1.68 7807.37 2.64191302119179451 +AAAAAAAANKCAAAAA Low, large clouds will not visit for example as the notions. Small, unacceptable drugs might not negotiate environmental, happy keys. Books business 3.11 3933.56 1.33106582416859905 +AAAAAAAAOAPAAAAA Silver, critical operations could help howev Books business 5.56 9087.69 3.07515674850230731 +AAAAAAAAOBAEAAAA Terrible, psychiatric bones will destroy also used studies; solely usual windows should not make shares. Advances continue sufficiently. As key days might not use far artists. Offici Books business 5.83 2024.32 0.68500370381562209 +AAAAAAAAOCHCAAAA Too white addresses end by the talks. Hands get only companies. Statements know. Sentences would pay around for a payments; papers wait actually drinks; men would Books business 6.06 5178.86 1.75245923645598158 +AAAAAAAAAGLDAAAA New, big arguments may not win since by a tenant Books computers 1.00 1686.72 0.45326198032962534 +AAAAAAAAALNBAAAA Else substantial problems slip months. Just unique corporations put vast areas. Supporters like far perfect chapters. Now young reports become wrong trials. Available ears shall Books computers 51.46 471.00 0.12656895793922734 +AAAAAAAABEBEAAAA Cheap, desirable members take immediate, estimated debts; months must track typica Books computers 3.26 7226.83 1.94202195818247621 +AAAAAAAABHOAAAAA Expert, scottish terms will ask quiet demands; poor bits attempt northern, dangerous si Books computers 2.66 4463.91 1.19955931429829364 +AAAAAAAACCDBAAAA Gradually serious visitors bear no doubt technical hearts. Critics continue earlier soviet, standard minute Books computers 6.40 3244.45 0.87186126451364360 +AAAAAAAACCPBAAAA Clear, general goods must know never women. Communications meet about. Other rewards spot wide in a skills. Relative, empty drawings facilitate too rooms. Still asian police end speedily comp Books computers 7.64 5385.04 1.44708896233770017 +AAAAAAAACDFEAAAA Wide, essential activities make steadily procedures. Modules Books computers 35.95 7285.54 1.95779873848101557 +AAAAAAAACFMBAAAA At least remaining results shall keep cuts. Clients should meet policies. Glorious, local times could use enough; clever styles will live political parents. Single, gradual contracts will describe ho Books computers 9.51 14393.77 3.86795004186180949 +AAAAAAAACLPDAAAA Environmental, new women pay again fingers. Different, uncomfortable records miss far russian, dependent members. Enough double men will go here immediatel Books computers 89.89 9487.37 2.54948308807619376 +AAAAAAAACOFCAAAA Years learn here. Days make too. Only moving systems avoid old groups; short movements cannot see respectiv Books computers 0.60 1706.66 0.45862033493962150 +AAAAAAAACONDAAAA Magnetic Books computers 57.19 7638.33 2.05260184394042112 +AAAAAAAADAHAAAAA Ga Books computers 5.53 7904.13 2.12402865714688954 +AAAAAAAADDBAAAAA S Books computers 65.78 578.19 0.15537347301673430 +AAAAAAAAEAHCAAAA Simple year Books computers 3.01 5038.44 1.35394925783295241 +AAAAAAAAECEEAAAA Agricultural players shall smoke. So full reasons undertake Books computers 0.70 5739.18 1.54225484506508439 +AAAAAAAAECGEAAAA Then basic years can encourage later traditions. For example christian parts subscribe informal, valuable gr Books computers 2.75 11359.94 3.05268740563088364 +AAAAAAAAECHAAAAA Boxes batt Books computers 0.83 6659.54 1.78957757569979198 +AAAAAAAAEFFAAAAA Blocks extend ev Books computers 9.29 11249.90 3.02311702743208836 +AAAAAAAAEIGCAAAA Separate, dead buildings think possibly english, net policies. Big divisions can use almost Books computers 9.46 3529.22 0.94838577014496795 +AAAAAAAAEJECAAAA Artists make times. Rather ready functions must pre Books computers 5.71 7805.93 2.09763996995021836 +AAAAAAAAFDPCAAAA Limited, capable cities shall try during the bodies. Specially economic services ought to prevent old area Books computers 2.93 6458.26 1.73548882866368225 +AAAAAAAAFGLAAAAA Since other birds shall blame sudden Books computers 6.74 2404.97 0.64627292308939187 +AAAAAAAAFHNAAAAA Legs throw then. Old-fashioned develo Books computers 2.66 12518.22 3.36394492707854445 +AAAAAAAAFJBEAAAA Only careful men define judicial, special lawyers. Now able funds will not put too black, economic terms. Objectives know both points. Teeth pay. Books computers 9.85 911.50 0.24494183686115864 +AAAAAAAAGHEAAAAA Men should not turn shadows. Different, single concessions guarantee only therefore alone products. Books computers 8.38 11864.77 3.18834729318175442 +AAAAAAAAGIFEAAAA Educational, white teachers should not fix. Considerable, other services might not cover today on a forms. Successful genes fall otherwise so Books computers 1.65 7042.80 1.89256869845942737 +AAAAAAAAGKKAAAAA Women note days. Other, efficient qualificati Books computers 7.64 8012.26 2.15308577269247054 +AAAAAAAAHGCEAAAA Present Books computers 2.84 4786.32 1.28619858760866792 +AAAAAAAAHHFDAAAA Multiple, dark feet mean more complex girls; schools may not answer frequently blue assets. Spiritual, dry patients may reply personnel Books computers 2.04 2973.19 0.79896721880112808 +AAAAAAAAIBDEAAAA Private teachers ap Books computers 5.27 8109.35 2.17917617635769258 +AAAAAAAAIDCDAAAA Daily numbers sense interesting players. General advantages would speak here. Shelves shall know with the reductions. Again wrong mothers provide ways; as hot pr Books computers 7.56 13142.36 3.53166626340166409 +AAAAAAAAIECAAAAA Inc, corporate ships slow evident degrees. Chosen, acute prices throw always. Budgets spend points. Commonly large events may mean. Bottles c Books computers 68.38 12405.10 3.33354687926095337 +AAAAAAAAINGCAAAA European, possible problems ought to restore then unfair interests. States would sleep in respect of the questions. Ideal stages affect only pressures. About spanish employees might kno Books computers 3.42 6760.19 1.81662463645686890 +AAAAAAAAIOJBAAAA Upper others narrow deaths. Situations could e Books computers 5.42 10932.74 2.93788855460829783 +AAAAAAAAIOKCAAAA However old hours ma Books computers 8.84 5208.75 1.39971562561772907 +AAAAAAAAJDOCAAAA Indeed other actions should provide after a ideas; exhibitio Books computers 6.95 3949.76 1.06139491997885895 +AAAAAAAAJJFCAAAA Effective females must answer too english projects. Firm, political experiments see in terms of Books computers 0.76 246.87 0.06633986973770075 +AAAAAAAAJPBDAAAA Of course responsible fears tell. Now clear substances might develop at least independent civil tourists. Books computers 4.95 619.44 0.16645833398274943 +AAAAAAAAKDGEAAAA Perfect days find at all. Crimes might develop hopes. Much socialist grants drive current, useful walls. Emissions open naturally. Combinations shall not know. Tragic things shall not receive just Books computers 6.71 8038.78 2.16021233057898500 +AAAAAAAAKEAAAAAA Advantages apply almost on a services; materials defeat today individual ideas. Domestic divisions used to win smoothly irish talks. Subsequent quantities make only, automatic pounds. Flower Books computers 7.87 442.26 0.11884583298981461 +AAAAAAAAKJDBAAAA Close, historic tactics lead ago large, typical stars. Generally significant facilities check leading years; yesterday general years Books computers 3.87 8448.38 2.27028164092273769 +AAAAAAAAKJECAAAA Internal seats used to sell dark words. Universal items show now in the roles. Most wonderf Books computers 2.57 870.24 0.23385428865612144 +AAAAAAAAKKACAAAA Likely, separate attacks prefer seats. Informally equal women could use easy prime, big forces. Long technical women save conditions; fast alone rooms sell. Ne Books computers 3.77 344.40 0.09254851191989362 +AAAAAAAAKOJBAAAA Economic customs should not put unpleasant shops. Colonial, middle goods used to see. Closely explicit legs continue Books computers 3.32 8481.54 2.27919252551990282 +AAAAAAAAKOKBAAAA Human windows take right, variable steps. Years should buy often. Indeed thin figures may beat even up to a cars. Details may tell enough. Impossible, sufficient differences ought to return Books computers 4.47 5466.16 1.46888784528468556 +AAAAAAAAKPNDAAAA Left diff Books computers 0.74 3269.32 0.87854442796151745 +AAAAAAAALBNAAAAA Male levels shall reduce else high, local conditions; further personal agencies control. Successful days wake eve Books computers 6.55 2376.38 0.63859010672531010 +AAAAAAAALEJBAAAA Wide governments conform widely in proportion to a friends. So living rooms wear too clothes; most essential measures should not bring previously pains. Real accounts become also gue Books computers 9.35 11110.42 2.98563541755233586 +AAAAAAAALIHCAAAA Places transform Books computers 3.10 5805.20 1.55999599708875273 +AAAAAAAAMADEAAAA Appropriate effects beg passages. Running contracts must keep only upper sons. Safely available reports intend perhaps w Books computers 5.81 8969.60 2.41034591323077181 +AAAAAAAAMDKBAAAA Friendly, hot computers tax elsewhere units. New, real officials should l Books computers 3.19 2999.57 0.80605615534133364 +AAAAAAAAMELCAAAA Perfect members state democratic schools. Genuine, enormous knees must afford around the implications. Matters will indicate with a months. Still regular machines would not l Books computers 4.07 2110.95 0.56726272136265806 +AAAAAAAAMENDAAAA Kinds relieve really major practices. Then capable reserves could not approve foundations. Pos Books computers 7.23 1522.62 0.40916438797755059 +AAAAAAAANANCAAAA Only increased errors must submit as rich, main Books computers 6.94 8287.27 2.22698753303826016 +AAAAAAAANFHDAAAA Meals ought to test. Round days might need most urban years. Political, english pages must see on a eyes. Only subsequent women may come better methods; difficult, social childr Books computers 7.23 15325.54 4.11833891222069241 +AAAAAAAANHFDAAAA Systems cannot see fairly practitioners. Little ca Books computers 1.73 2428.71 0.65265242852777245 +AAAAAAAANKLDAAAA Past beautiful others might not like more than legislative, small products. Close, wh Books computers 3.02 4174.86 1.12188467036552578 +AAAAAAAAOGDDAAAA Main problems wait properly. Everyday, foreign offenders can worry activities. Social, important shoes will afford okay physical parts. Very Books computers 1.40 939.26 0.25240161238640906 +AAAAAAAAOGMDAAAA Schools offer quickly others. Further main buildings satisfy sadly great, productive figures. Years contribute acti Books computers 4.11 885.92 0.23806787944271822 +AAAAAAAAOKLAAAAA Chief me Books computers 2.62 9675.59 2.60006230094948754 +AAAAAAAAOMDAAAAA Tiny, rare leaders mention old, precious areas; students will improve much multiple stars. Even confident solutions will include clearly single women. Please little rights will not mention harder com Books computers 1.45 3092.13 0.83092923972956056 +AAAAAAAAONDCAAAA Guidelines should investigate so. Usual personnel look now old, modern aspects. Discussions could appear once br Books computers 2.44 9923.72 2.66674076280396839 +AAAAAAAAONHDAAAA Flat pleasant groups would go private, redundant eyes. Main devic Books computers 2.83 2445.21 0.65708637291417851 +AAAAAAAAOPDAAAAA Popular, obvious copies should believe still difficult parts. Forms ought to soften characteristic Books computers 1.05 2156.19 0.57941979069847684 +AAAAAAAAPAKAAAAA Real, domestic facilities turn often guilty symptoms. Winds get naturally intense islands. Products shall not travel a little clear shares; improved children may not apply wrong c Books computers 5.28 1338.00 0.35955258115219995 +AAAAAAAAAADCAAAA Directions would ask yet profits. Forthcoming, specified discussions ought Books cooking 0.58 5750.02 2.05632295473197783 +AAAAAAAAABIBAAAA Then irish champions must look now forward good women. Future, big models sign. Then different o Books cooking 85.81 2279.71 0.81527020830049933 +AAAAAAAAACLDAAAA Black ears see sensibly glad months. Equal members must afford approximately o Books cooking 8.37 10363.44 3.70617485886789408 +AAAAAAAAAEFBAAAA Considerable benefits should govern. Well experienced years provide please in an towns. Exc Books cooking 4.18 0.00 0.00000000000000000 +AAAAAAAAAGHBAAAA Valuable studies should persist so concerned parties. Always polite songs include then from the holes. There conventional areas might not explain theore Books cooking 1.58 1326.45 0.47436523408687831 +AAAAAAAAAIJCAAAA Meanings occur in a things. Also essential features may not satisfy by the potatoes; happy words change childre Books cooking 3.46 1262.55 0.45151330717055917 +AAAAAAAAAJDBAAAA Then dominant goods should combine probably american items. Important artists guess only sill Books cooking 6.67 5569.20 1.99165808110116677 +AAAAAAAACDIBAAAA Libraries shall note still. Children would not concentrate. Local, public modes must mean low children. Outer, good years should vis Books cooking 1.42 2178.99 0.77925070784648269 +AAAAAAAACJJAAAAA Children ought to miss historical effects. Honest girls may not think activities. Woo Books cooking 4.42 348.88 0.12476651428114901 +AAAAAAAACLFEAAAA Single, past rates mark blue, evident discussions. Only literary children ought to publish exactly really recent themes; conscious, ready conditions would adopt advanced, ideal provisions. A Books cooking 4.99 9499.97 3.39738059698316657 +AAAAAAAACPEEAAAA Standards could lead no longer ago great tactics; difficult lives might feel french, easy costs. Students drop certainly unabl Books cooking 3.05 16321.01 5.83672187356046718 +AAAAAAAADDNAAAAA Individual, remarkable services take by the interest Books cooking 6.05 1054.65 0.37716408016112647 +AAAAAAAADIGCAAAA Positions shall Books cooking 4.21 2629.53 0.94037288551281172 +AAAAAAAAEGFEAAAA Ultimately senior elections marry at l Books cooking 5.06 7756.87 2.77401293175881769 +AAAAAAAAEINDAAAA Hence young effects shall not solve however months. In order small activities must not return almost national foods. International decades take contributions. Sessions must see Books cooking 1.43 19276.07 6.89351084309627374 +AAAAAAAAEKJCAAAA Members need for a regions. Leading needs go at least under the others; old police could play on a drinks. Very similar machines must consider fully effec Books cooking 9.86 10833.86 3.87440652490818908 +AAAAAAAAEKPDAAAA Mainly catholic activities could assume just fat, o Books cooking 2.68 2262.61 0.80915490391444210 +AAAAAAAAENDCAAAA Points trace so simple eyes. Short advisers shall not say limitations. Keys stretch in full now blue wings. Immediately strategic students would not make strangely for the players. Books cooking 1.69 5132.94 1.83564271902740482 +AAAAAAAAEPJDAAAA Projects become more from a pupils. Details may precede generally; good, marvellous birds could suffer fair Books cooking 9.88 628.36 0.22471419087853357 +AAAAAAAAFDIAAAAA Great pp. will not r Books cooking 1.91 2941.23 1.05184308300603044 +AAAAAAAAFPEBAAAA New, general students raise therefore to a women; united letters would start black positio Books cooking 4.03 3747.49 1.34017789670793138 +AAAAAAAAGFGCAAAA Products may not resist further specif Books cooking 5.37 8721.33 3.11892325153523644 +AAAAAAAAHBGCAAAA Dramatic months deal broadly in a films. Almost new occasions may get together sources. Under dry orders wor Books cooking 3.92 1412.78 0.50523858073297895 +AAAAAAAAHCGEAAAA Thus certain stars appear totally even local guests. Urban friends might not take properly various vehicles Books cooking 4.55 1446.44 0.51727607462974425 +AAAAAAAAICAAAAAA Somet Books cooking 7.34 6593.72 2.35804706645808830 +AAAAAAAAIFHDAAAA Genetic properties might describe therefore leaves; right other organisers must not talk even lives; methods carry thus wrong minutes. Proud worke Books cooking 1.08 119.92 0.04288580713309846 +AAAAAAAAIHHDAAAA Urgent agencies mean over as a plants; then Books cooking 6.47 9566.59 3.42120525067902230 +AAAAAAAAIJJAAAAA Men could require evolutionary falls; taxes teach dead parents; only financial servants might not buy eastern things. Different payments develop. New inhabitants might not eat w Books cooking 80.50 3855.42 1.37877583836799906 +AAAAAAAAINMBAAAA Hours ought to cope thus into the eyes. Dark states reduce most for the feelings. National, tragic children shall establish enough typical boats. In order secret hours must mean; sin Books cooking 2.30 12966.63 4.63712802990534045 +AAAAAAAAJHGDAAAA Guests agree around trying, young costs. Here annual banks appeas Books cooking 58.88 8031.52 2.87223330308224573 +AAAAAAAAJIHBAAAA Wonderful qualities suffer of course light leaders. True clubs used to see early living operat Books cooking 9.91 4482.62 1.60307518988467144 +AAAAAAAAKABAAAAA High big appeals may Books cooking 36.23 675.62 0.24161531867298181 +AAAAAAAAKCGEAAAA Final women should establish on a pupils. Full, northern years might not avoid full Books cooking 60.38 5877.02 2.10174071245298770 +AAAAAAAAKECDAAAA Little part Books cooking 9.90 4729.36 1.69131438311366337 +AAAAAAAAKFIAAAAA Here other affairs afford directly effective leads. Plants cannot undertake as coming, huge photographs; d Books cooking 0.87 20785.39 7.43327407210001090 +AAAAAAAAKFNBAAAA Stairs might bring early others. Large forms rel Books cooking 1.88 2350.18 0.84047169953356678 +AAAAAAAAKHADAAAA Now available m Books cooking 3.55 1102.96 0.39444070910208700 +AAAAAAAAKJNDAAAA Major instructions put flatly british, other democrats. Operations represent well upon a stores. Thousands will not appear surely Books cooking 1.29 582.88 0.20844962693245854 +AAAAAAAAKLACAAAA New, single products raise too extreme, efficient minutes; hands support leaders. Additional, english things prefer halfway private, slow churches. High white things could f Books cooking 4.13 2472.08 0.88406559454294555 +AAAAAAAAKLEDAAAA Golden, sure days fill of course. Early free minutes must not express only, cap Books cooking 9.44 4521.21 1.61687575106934680 +AAAAAAAAKLOCAAAA Purposes hide tears. Small laws award good eyes. Books cooking 55.11 5382.78 1.92499053468895684 +AAAAAAAALCJDAAAA Yet religious animals ensure also. Rough, real heads resist dead. Civil, evolutionary votes dissuade rapidly left cars. Buyers Books cooking 2.20 11624.81 4.15726617427380135 +AAAAAAAALLNCAAAA Here comprehensive years should tend sensibly particular front sales. Official, coherent tears regulate animals. Rewards cannot w Books cooking 2.50 2499.59 0.89390372458156745 +AAAAAAAAMAODAAAA Widely splendid others deprive only. Different, main soldiers discover then other periods. Too main birds must change public, terrible houses. Different, armed females may foster; science Books cooking 4.26 6853.89 2.45108909816104214 +AAAAAAAAMGBDAAAA New women add however. Scottish managers place mostly. Normal, financial purposes should lea Books cooking 4.74 319.20 0.11415234853973505 +AAAAAAAAMINCAAAA Extra theories drop. Other resources shall know eventually anyway open students. Long-term, liable di Books cooking 6.96 5834.64 2.08658477093947276 +AAAAAAAAMJIAAAAA Special, public skills agree at a parent Books cooking 5.87 4713.66 1.68569974692295585 +AAAAAAAAMKGAAAAA Gentle fans cannot pay else can Books cooking 2.45 7576.48 2.70950183478800689 +AAAAAAAANHBBAAAA Sound, new offices might equip hot, new reports; calculations fight great scientists. Professional, little issues learn of c Books cooking 66.16 6628.48 2.37047794250834265 +AAAAAAAAOLDBAAAA Well angry rebels drop in a methods. Studies argue most sometimes residential organisations. Rural, different children know o Books cooking 4.42 453.06 0.16202338041795852 +AAAAAAAAPAFCAAAA Half general features used to defend as ready medical pounds. Turkish, trying rooms secure with a ci Books cooking 7.08 683.53 0.24444409397670770 +AAAAAAAAPHHCAAAA African, elected carers would examine proba Books cooking 6.20 15598.69 5.57840569437117702 +AAAAAAAAAFBEAAAA Already accessible clubs match all enough o Books entertainments 5.00 1196.30 0.46493128593083651 +AAAAAAAAAGKBAAAA Likely, various days develop no longer. Officials say before agricultural, rare ho Books entertainments 2.67 23516.84 9.13960934734576042 +AAAAAAAAAIKBAAAA Less progressive experiences would silence as economic, soviet specialists. Alone legal brothers fight only ears. Methods could not return records. E Books entertainments 8.36 5931.28 2.30513887621487248 +AAAAAAAAAJCDAAAA Strict heads discuss as categories. Alone, specific markets wait single, human numbers. External, various changes want very relatively nuclear orders. Old, pre Books entertainments 2.32 4525.09 1.75863572068274594 +AAAAAAAAAMFCAAAA Instances used to lower out of a costs. Irish supporters sign in a types. Bad things shall participate clear Books entertainments 1.58 3570.57 1.38767006737947580 +AAAAAAAAANJDAAAA Trustees may encourage today necessary, political tears; inner, foreign times pay in the historians. Areas may belie Books entertainments 1.79 17322.75 6.73233171726021741 +AAAAAAAABAJDAAAA Rare, radical beds say over readers; han Books entertainments 7.10 7808.46 3.03468807902658165 +AAAAAAAACKAEAAAA L Books entertainments 1.63 4264.23 1.65725481685601518 +AAAAAAAACLMDAAAA Always constitutional advertisements go for a spaces. Cars spend bad difficulties. Rights encourage further great qualities. Blue, high homes would produce u Books entertainments 2.63 3974.52 1.54466161878945775 +AAAAAAAACMOAAAAA Companies ought to record now detailed, good roads. Muscles shall not argue significantly territorial months. Clearly new periods could write in a committees. Figures will not find more there Books entertainments 3.07 7276.45 2.82792715498740725 +AAAAAAAACMPAAAAA Falsely large trees shall reflect against a Books entertainments 0.70 957.09 0.37196446079707792 +AAAAAAAAEGNDAAAA Deep patterns shall find british, american expectations. Sufficient patients must see. English, large assets could not meet for the proceedings. White, chinese matches shal Books entertainments 0.56 1499.01 0.58257681762365897 +AAAAAAAAEKGAAAAA Particular, deliberate things rain however original ways. Virtually old deaths consider women. Notably w Books entertainments 9.71 1611.84 0.62642718708915783 +AAAAAAAAEPCEAAAA New, previous police outline right in a persons. Wealthy quest Books entertainments 2.07 5037.58 1.95781037146155928 +AAAAAAAAFAKBAAAA Doors cannot happen here severe, old rates. Inevitable, int Books entertainments 2.29 1047.84 0.40723363591888968 +AAAAAAAAFGJCAAAA Limitations respond. Bare rivers will not create yesterday. Well local persons may unders Books entertainments 8.95 2096.28 0.81470045646668390 +AAAAAAAAGCBCAAAA So perfect changes would twist again; legal standards like improvements; rights used to tell working stations. Official, immediate loans listen much possible pictures. Always d Books entertainments 6.32 1017.52 0.39545003933824690 +AAAAAAAAHFABAAAA Prisons take angry, logical sums. Now old grounds cannot help so increased problems. Blue, negative designs would freeze. Small payments ask alike to a hundreds. Exte Books entertainments 2.62 11202.91 4.35391068500161131 +AAAAAAAAHGMAAAAA High, official employees shall not start too left circumstances. Patients used to touch obviously popular, senior members. British, impossible theories make only. Young, international wo Books entertainments 4.85 1041.70 0.40484737988309988 +AAAAAAAAHPPDAAAA Now old tears give. Other kids coincide up a animals; specific procedures remove future, french levels. Coming, strong values a Books entertainments 5.08 24460.84 9.50648649682223761 +AAAAAAAAIACBAAAA Large women establish today polite, easy horses. Details sha Books entertainments 5.06 1748.58 0.67956996401650263 +AAAAAAAAIDDAAAAA Plans would not claim; most particular horses will not tell simply cases; more british enquiries could not smile blue men. Old, dangerous play Books entertainments 0.95 6942.27 2.69805108950854163 +AAAAAAAAIKFAAAAA Pieces threaten Books entertainments 0.69 1273.35 0.49487607869266126 +AAAAAAAAINCCAAAA Cases can accept gmt sudden services; tools show all also capable meals; important, spatial days would not happen human, cold backs. Red, economic effects must s Books entertainments 9.58 1334.73 0.51873086622959576 +AAAAAAAAIPLAAAAA Financial gods might presume divine, tiny Books entertainments 8.42 731.84 0.28442306469583164 +AAAAAAAAKBFEAAAA Marginal, available teeth pay recently little services. Then british times could require more scottish fair tea Books entertainments 95.74 3018.65 1.17317130007115240 +AAAAAAAAKFGCAAAA Now complete others shall pass. Just front advantages could exercise more establish Books entertainments 6.51 5281.66 2.05266987849992639 +AAAAAAAAKKIDAAAA Young reasons could not say impossible experiences. Prisoners cancel particularly; forms might k Books entertainments 3.77 3626.88 1.40955444480216694 +AAAAAAAAKLNAAAAA Just particular actions seem very; necessarily poor eleme Books entertainments 0.26 6872.96 2.67111437845958545 +AAAAAAAAKMOCAAAA Japanese, efficient sports withdraw recently severe days; factors mean originally impossible items. Quiet workers become from a officers. Pieces explore. For example o Books entertainments 3.74 16796.75 6.52790652592057016 +AAAAAAAALMJCAAAA Never able feet go on the provisions. Services play brown studies. Cruel, Books entertainments 9.79 12846.63 4.99272774870656373 +AAAAAAAAMDOCAAAA Internal claims speculate perhaps through a expectations. Immediate courts appeal to a councils; transactions materialise entirely; fine, serious conditions may not use to a types. Short, large lip Books entertainments 3.11 5231.34 2.03311346095579892 +AAAAAAAAMJABAAAA Front, possible foundations hear well. Old, close inches change pointedly for the employees; odd, financial members work under on the arrangements; st Books entertainments 92.23 225.66 0.08770073893099771 +AAAAAAAAMJDCAAAA Local words co Books entertainments 2.95 9381.26 3.64594271959501737 +AAAAAAAAMOFBAAAA Hardly local women should tell easily tall, able issues. Important, available conditions could no Books entertainments 2.21 15740.54 6.11741996442846214 +AAAAAAAAMOHBAAAA General, raw tests would not buy heavy, considerable blues. High, regional modules meet often young, responsible calculations. Things hesitat Books entertainments 2.00 5567.90 2.16391449212931922 +AAAAAAAAOAODAAAA H Books entertainments 4.80 2493.52 0.96908422644341674 +AAAAAAAAOBODAAAA New hours borrow new poets. Youngsters mind especially. Laws must add there in a ends. Factors must not take strategic, royal tr Books entertainments 2.30 4109.90 1.59727584389128560 +AAAAAAAAOEGCAAAA Clear materials will ship evidently literally scottish targets. Residential heads make prominent times. Internal, open subjects feel subsequent Books entertainments 0.75 263.40 0.10236805208909332 +AAAAAAAAOEMBAAAA Other practices get feet. Numbers will not increase now large, simple foreigners. Flowers cover Books entertainments 1.00 315.51 0.12262013710945267 +AAAAAAAAOFEAAAAA Heavy, formal factors could want then truly serious players. Be Books entertainments 4.31 8757.62 3.40357061631163789 +AAAAAAAAPCKBAAAA Men call tonight particularly mental lines. Recent markets must dress children. Multiple relations should seem relatively about a arts. Funny, real proteins shall keep citie Books entertainments 5.20 3090.94 1.20126616144366780 +AAAAAAAAPEGAAAAA Dirty trials should get. Balls shall win later national programmes. Elements ought to explain apart poss Books entertainments 1.62 290.34 0.11283804192690719 +AAAAAAAAPFIAAAAA Subsequent, Books entertainments 1.29 9603.95 3.73248919461293761 +AAAAAAAAPNPAAAAA Countries turn more actually scientific patients. Good writers could not drag perhaps. Suddenly left months cannot announce more overall loans; beds transform far Books entertainments 1.32 2401.56 0.93334479565331415 +AAAAAAAAACCEAAAA Royal, blue men used to convey jobs. Other, technical things would say as mere children; ab Books fiction 0.62 555.50 0.18274906106295868 +AAAAAAAAAEGEAAAA Exclusively ready fields invest right in a courts. Quite glad facts would not imitate usually by a types. More large managers can continue both small matters. Additional, basic scholars s Books fiction 1.11 3969.44 1.30587116641899316 +AAAAAAAAAIDCAAAA Dollars get on a years; separate economies can say. Firms know even sons. Simple, definite members will say most cold, big policies; main, true agents might repeat too. Elements know goods. Great Books fiction 5.03 149.04 0.04903135924540659 +AAAAAAAAAMOCAAAA Wild officials will not watch onl Books fiction 0.47 6954.51 2.28790310108543073 +AAAAAAAAAODAAAAA Just minor eyes exc Books fiction 7.11 16681.12 5.48777500896227056 +AAAAAAAABGFEAAAA Married circumstances face human, compulsory hours. Years make sometimes national problems. Difficulties should invest far running, medical centuries; perf Books fiction 2.71 10221.52 3.36268799754501063 +AAAAAAAABJAAAAAA Other horses apply able schools; possible enquiries would not describe easily r Books fiction 3.83 10067.63 3.31206107944063852 +AAAAAAAABKFAAAAA Firm, local examinations may not sponsor most rural charges. Countries shall help beautiful, different terms Books fiction 7.72 5090.34 1.67462620250444840 +AAAAAAAABOJBAAAA As joint men would so Books fiction 2.13 2773.11 0.91230107781152357 +AAAAAAAABPGDAAAA Pictures get with a conditions; still gross eyes go that. Personal beings contact thereafter in a systems. New, medium goals might not tell; as official years mu Books fiction 5.52 2061.58 0.67822107885899974 +AAAAAAAACEBDAAAA Essential, alternative fans let unlikel Books fiction 1.52 2460.17 0.80934969856932323 +AAAAAAAACMNBAAAA Basic changes may not see; afraid names seek in excess of a characteristics. Awful scientists shall not want now right eyes. Here used workers will not pray in part Books fiction 2.27 6034.24 1.98515156476786280 +AAAAAAAACNBBAAAA Local companies would restrict yet most imaginative days. Married, str Books fiction 99.71 7003.69 2.30408239689654919 +AAAAAAAADCGCAAAA Different stations may smell; weapons disguise cons Books fiction 1.47 1671.19 0.54979010505455611 +AAAAAAAADCODAAAA Private, quiet populations shall receive more somewhat proposed machines. Heads protect abroad parent Books fiction 74.86 3243.16 1.06693869464796593 +AAAAAAAADDBDAAAA Circumstances should include parties. Good investigations fall as deposits. Characters might force at all convenient, special years; Books fiction 5.18 12.59 0.00414187340914968 +AAAAAAAADNMAAAAA Old, official cases look enough; actual emotions go statistical, wild limits. Mental cities hear above mod Books fiction 2.55 769.44 0.25313130070978025 +AAAAAAAADPHDAAAA Times should not get on a lists; different students undermine suddenly groups. Even actual modules may stay for a Books fiction 8.31 638.38 0.21001502358482729 +AAAAAAAAEBJAAAAA Techniques render eventually dark tiles. Only, other centres would bid at the falls. Sorry, full days write for a groups. Both Books fiction 2.99 6665.04 2.19267291079579140 +AAAAAAAAEEEEAAAA Towns see even afraid, mean factors. Soldiers spend areas; resu Books fiction 48.40 9444.91 3.10719790157362568 +AAAAAAAAEJDEAAAA Loud young standards remove enough green values; important students cannot receive particular police; significant authorities should not expect Books fiction 52.22 8870.17 2.91811924206809036 +AAAAAAAAFHDBAAAA Good, bad cats could not finance libraries. Concerned names get at Books fiction 0.13 5959.16 1.96045165566866039 +AAAAAAAAFMHBAAAA Years take critics. Again academic areas look high under a w Books fiction 90.57 742.90 0.24440013944855446 +AAAAAAAAGFHBAAAA Ambitious, isolated mines should Books fiction 9.67 5292.65 1.74118239070183305 +AAAAAAAAGFODAAAA Wives must file upon a respects; anywhere growing wounds may not develop yet for a demands; quite key sides could not make fresh men. Dead times Books fiction 18.03 6121.11 2.01373016230978759 +AAAAAAAAGIOAAAAA Thus separate stars will touch lightly commercial great institutions. Personal, brief hands will not concern always smart rules. Dead Books fiction 4.96 2769.10 0.91098186316730672 +AAAAAAAAHDHCAAAA Difficult decisions retain concerns. Accordingly parliamentary cases worry only inadequate, good scores. Responsible adults exist still well silly Books fiction 2.74 2397.93 0.78887390818127904 +AAAAAAAAHFFCAAAA Necessarily royal losses ought to say courses. True, current Books fiction 0.62 5056.32 1.66343426180712733 +AAAAAAAAIBMCAAAA Others reflect much up to a paintings; twice narrow cases cannot wear however hard major wings. Popular bacteria go Books fiction 8.71 3061.36 1.00712991102736127 +AAAAAAAAIFBAAAAA Usually sure students give. Popular resources may give especially full, fine paintings. Ever possible borders shall not free. New bodies help apart. Further main readers could esca Books fiction 3.51 11100.42 3.65182958128620664 +AAAAAAAAIGADAAAA Communications move afterwards different errors; warm goods give at all. Twins could return f Books fiction 0.34 5726.99 1.88407208859937665 +AAAAAAAAIJPDAAAA New, united books ought to earn things. Home domestic bands shal Books fiction 3.36 8480.61 2.78996132266631505 +AAAAAAAAIMOAAAAA Different, expensive years used to learn humans. Normally parliamentary cards benefit. Certain consequences used to encourage. More new proposals could not prom Books fiction 3.33 8887.28 2.92374811053755431 +AAAAAAAAIOMDAAAA Good levels ask quiet, particular objects. Previously rural re Books fiction 4.72 3395.05 1.11690765033626979 +AAAAAAAAJHGAAAAA Large hearts used to say annually. For example separate criteria should admit gay ministers. Growing, ordinary Books fiction 1.92 3430.77 1.12865885908724888 +AAAAAAAAJHIBAAAA Plans mi Books fiction 4.76 533.80 0.17561016884861808 +AAAAAAAAJJBAAAAA Citizens can b Books fiction 4.61 584.00 0.19212502549193136 +AAAAAAAAJKFEAAAA Personal, sympathetic text Books fiction 0.15 3428.40 1.12787917362420799 +AAAAAAAAJLODAAAA Social, private books ought to demand merely social companies. Alive, swiss police will rest again victorian, married commentators. Standard, european territories attend to a comments. Books atte Books fiction 2.81 3504.94 1.15305939528714023 +AAAAAAAAJOKBAAAA Favourably present words can make small, economic cases. About eastern years give less views. Only possible workers may accept even requirements. Negative goods imp Books fiction 4.00 4392.10 1.44491836380669814 +AAAAAAAAKDGAAAAA Provinces complement more. Participants cannot lie swiftly then total muscles. Unions surprise perio Books fiction 2.17 1757.38 0.57814499537501769 +AAAAAAAAKGGBAAAA New, novel individuals used to pay at the rates. Especially social values sleep too unaware cattle. Also immediate changes give almost chains. Swee Books fiction 1.98 11006.58 3.62095798472428397 +AAAAAAAAKHNCAAAA Also good forms Books fiction 4.30 2992.89 0.98460456771326445 +AAAAAAAAKIJCAAAA Mo Books fiction 6.72 9516.74 3.13082862174671717 +AAAAAAAAKMMAAAAA Then wild sciences will know in a chemicals. Extremely Books fiction 5.84 10044.66 3.30450438109209457 +AAAAAAAAKNJAAAAA Likewise high penalties might afford never square, thin Books fiction 1.65 209.10 0.06878997059993638 +AAAAAAAALDNBAAAA Enough little accountants light only important, great systems. Determined sk Books fiction 0.36 6117.14 2.01242410691389210 +AAAAAAAALGFAAAAA Primary, good features assess then early, bad c Books fiction 4.63 2352.74 0.77400724739021675 +AAAAAAAALHNDAAAA Mass attitudes may like occupational state Books fiction 6.40 528.87 0.17398829149300982 +AAAAAAAALIOBAAAA Additional officers shall not apply so poin Books fiction 9.09 6890.24 2.26675947884507726 +AAAAAAAAMAEAAAAA In order financial glasses must kill convenient, important papers. Shy cities like below fragments. Patients ma Books fiction 6.94 8176.49 2.68991155767897573 +AAAAAAAAMBOBAAAA Goods keep points. Again sensitive windows must not cause closely female, individual powers; gaps derive suddenly sincerely other hands; other houses may not imagine under for a data Books fiction 7.80 6049.19 1.99006983382797303 +AAAAAAAAMFPDAAAA Pretty realistic facts may not work without a guidelines. Overall patterns t Books fiction 15.95 13032.24 4.28736205859069780 +AAAAAAAAMOODAAAA Mechanically whole rooms might like then please specialist relatives. Als Books fiction 3.90 6774.40 2.22865029570640375 +AAAAAAAANGNCAAAA Important enterprises could flow without a countries; ugly, previous things see even de Books fiction 0.82 887.04 0.29181949077459382 +AAAAAAAANIEDAAAA Excellent, relevant concentrations seem exciting, local children. Units should not reinforce current lips; pure feet shall show always into a minutes. Commonly primit Books fiction 2.70 4113.69 1.35332670567791628 +AAAAAAAANLEDAAAA Conservative, available Books fiction 2.01 2510.09 0.82577244047438695 +AAAAAAAANOAEAAAA Black women treat really users. Expert, hard authorities should produce good indians; little, other details could waste. Ideas shall build. Low day Books fiction 0.72 9472.17 3.11616592930463604 +AAAAAAAANOBAAAAA Houses appear again scientific tests. Naked pieces shall not believe experiences. Coming, good measu Books fiction 1.86 2113.81 0.69540376735462230 +AAAAAAAAPFODAAAA Rates should not turn exactly enormous flowers. Happy practitioners should believe suddenly natural organisms; al Books fiction 2.51 3437.58 1.13089922111396129 +AAAAAAAAPJKAAAAA Constitutional, good pupils might not begin below level devices. External savings fit hardly. Parents shall dry. Actually literary companies improve a Books fiction 4.22 439.55 0.14460368999140142 +AAAAAAAAPPHDAAAA Eyes come no longer. Commercia Books fiction 0.20 5344.48 1.75823348671424196 +AAAAAAAAAAJBAAAA Famous authorities will demand at last growing teachers. Over immediate schools should go only so Books history 2.40 4151.41 1.32043953348399043 +AAAAAAAAAFCDAAAA Civil, english books could search either young institutions; incidentally major difficulties could not clinch little nevertheless old papers. Special subjects sail late workers. Low, national part Books history 1.01 1167.75 0.37142639855517278 +AAAAAAAAALGDAAAA At first close areas may Books history 0.09 9795.83 3.11576095719008192 +AAAAAAAAALPCAAAA Onwards current types may allow; other sectors might carry nowadays marginal conditions. Minutes add well faces. Urban, possible women could not oppose never markets; galleries must favour gently vehe Books history 59.17 3685.92 1.17238106697707767 +AAAAAAAAANBDAAAA Weapons wi Books history 3.85 1690.46 0.53768483810882242 +AAAAAAAAAPFDAAAA Odd, only premises present previously obvious strengths. Widely different times should not ke Books history 1.88 8472.00 2.69469017217677053 +AAAAAAAABAJAAAAA All female calls see ever fresh, widespread lawyers. Results could not want initially Books history 1.77 439.46 0.13977910092832903 +AAAAAAAABHDCAAAA Logical suggestions should evacuate in common equivalent, distinctive women. Fruits know formal pensioners Books history 1.85 10800.83 3.43542144149575407 +AAAAAAAABJCEAAAA Regular, elderly circumstances should not stop sole, different sites. New group Books history 2.98 383.28 0.12190992082057514 +AAAAAAAABKDCAAAA Also quiet users fall. Other, current sources would c Books history 0.43 10191.59 3.24164039327845288 +AAAAAAAABLLCAAAA Similarly legislative games could expect others. Central, special miles get all to a problems. Rights pass different, glad eyes. Most local tanks Books history 9.29 367.56 0.11690985831979388 +AAAAAAAACAJDAAAA Military areas used to help sometimes sooner certain children. Unlikely proceedings say; wages recognize now managerial years. New events stay full, royal communities Books history 6.86 9156.39 2.91237419093692870 +AAAAAAAADANAAAAA Wildly sexual powers achieve local, comfortable songs; artistic, very shares might start. Miners used to sleep very italian partners. Book Books history 4.58 3997.52 1.27149172061851791 +AAAAAAAADJPBAAAA Architects influence around enough visual interests. Days think already other issues. Regardless lucky rules mean to a shoulders. Women accept only. Books history 1.44 5541.90 1.76271287360557656 +AAAAAAAADNIBAAAA Never possible applications will not contribute still bad, golden resources; force Books history 5.60 5573.65 1.77281160034856670 +AAAAAAAAEIMBAAAA Armed profits forget now s Books history 9.04 494.12 0.15716481443295395 +AAAAAAAAEJAAAAAA Hundreds go over electronic fa Books history 7.68 898.62 0.28582418348931652 +AAAAAAAAEJCCAAAA In short new acres marry perfectly for a c Books history 1.58 186.93 0.05945685008085502 +AAAAAAAAEMGBAAAA Hostile, certain contents would carry; others can get great, prime rates. Expensive, national shows produc Books history 1.95 3076.78 0.97863182577314023 +AAAAAAAAFLAAAAAA Origins help still already common hands. Probably official increases could inform more recent, Books history 34.26 5002.56 1.59116492772953555 +AAAAAAAAGBGEAAAA Safe films go behind amo Books history 4.48 6872.36 2.18589246360490448 +AAAAAAAAGEBCAAAA Ancient, yellow sets anger other men. Beautiful, vari Books history 3.24 2349.53 0.74731532108527947 +AAAAAAAAGEDDAAAA Wheels shall include tables; only central days shall see lovely, jewish artists. Genes ought to climb therefore; Books history 2.02 6800.22 2.16294688416429633 +AAAAAAAAGMEAAAAA Branches attend fair true banks. Rigid cigarettes like by a places. Stations shall not let thus. Kids hold into a achievements. Streets used to set twice actual, wonderful areas; surroundings r Books history 6.21 12377.05 3.93676994753783023 +AAAAAAAAHIICAAAA Then sp Books history 1.91 8909.36 2.83380132582446085 +AAAAAAAAHKBEAAAA Parliamentary pieces shine never tragic patterns. Great human eyes would not get groups. Plant Books history 6.03 953.70 0.30334348645006918 +AAAAAAAAHOEDAAAA Tropical, different relations would not work eyes. Level customs might aff Books history 0.31 10335.72 3.28748384163962355 +AAAAAAAAHONAAAAA Ready, imperial shops see impossible assumptions. Clinical holders ask. Other rules would not avoid at a panels. Unusual, particular rights cannot go yet golden substance Books history 4.56 2768.79 0.88066940531413131 +AAAAAAAAIAPCAAAA Very valid police should not like away pictures. New, special principles survive from a Books history 4.76 8944.55 2.84499421382716393 +AAAAAAAAIEHBAAAA Fully classical offices cannot go different, new roads; proceedings mean asian, only groups. Earlier academic affairs Books history 3.37 10650.60 3.38763776531939474 +AAAAAAAAIHCCAAAA Big, special things find however happy agencies. Current firms reduce especially at a eyes. Imports want reasons. Little controversial de Books history 4.36 1262.68 0.40162079634137920 +AAAAAAAAIJBEAAAA Additional, human standards should not dream also silly forms. More independent friends may believ Books history 4.39 5255.61 1.67165257504650106 +AAAAAAAAIKGCAAAA Confidential, full terms make incorrectly elderly, real methods; teeth slip much today unknown conditions. Years shall not undermine occasionally local, industrial lips; restrictions beat most things Books history 1.38 7182.03 2.28438924188842437 +AAAAAAAAIKOAAAAA Independently mean findings must not take today police. White, yellow features try even grateful examples. Sweet Books history 2.06 4957.80 1.57692810854792173 +AAAAAAAAINEEAAAA Films cope Books history 1.22 14315.87 4.55345068403685835 +AAAAAAAAINKAAAAA Hours used to use always local, upper budgets. Only royal strategies confuse already key windows. Open, short habits broadcast just. Working-class lights will display previous measures. Soviet scho Books history 0.75 4671.20 1.48576920824741861 +AAAAAAAAJENCAAAA Opponents bring also waiting, other things. There massive characters contact Books history 58.48 1594.66 0.50721371930635138 +AAAAAAAAJGICAAAA Boys form so go Books history 4.24 12750.46 4.05554051613940340 +AAAAAAAAKADBAAAA Tomorrow soft actors could not go for the needs. Considerable times used to allow following visitors; months must not avoid about economic farmers. Tears start at firs Books history 1.76 10852.02 3.45170345163665691 +AAAAAAAAKCACAAAA Years would land in a trees. Areas establish above civil tests. Within so-called thanks like just. Ill acts prevent. Most Books history 8.83 11890.89 3.78213697136863066 +AAAAAAAAKDBAAAAA Allegedly great plans respond able, cheap facts. Today local banks might allow at least tr Books history 7.32 75.87 0.02413198103907597 +AAAAAAAAKMCEAAAA Effects shall not come in short southern firms. High, afraid years smell anyway governors. Wages can think deep, educational participants. Quietly probable Books history 88.42 7756.02 2.46695831789500422 +AAAAAAAAKOPBAAAA Particularly particular contents destroy feet. Essential, fatal wo Books history 2.76 7308.24 2.32453287345481131 +AAAAAAAALGCAAAAA Popular, current dogs shall not nominate respectively. More labour connections take further feet; holy, neighbouring months can leave. Attempts should investigate Books history 0.64 2234.94 0.71086766447176010 +AAAAAAAALKABAAAA Green discussions might offer most. Grateful feet ought to go still Books history 47.36 12676.50 4.03201604905557503 +AAAAAAAAMEPDAAAA Major, grateful charts talk system Books history 3.78 1685.71 0.53617400497404436 +AAAAAAAAMMEDAAAA Forward slight interests provide on a cases; successful areas must come internal, present en Books history 4.36 1180.89 0.37560584011116933 +AAAAAAAANCDEAAAA Soon sure forests cope; guilty, e Books history 6.82 3323.19 1.05700748740275284 +AAAAAAAANHACAAAA Grey words need. English, swiss measures help separat Books history 3.59 4100.58 1.30427202859119708 +AAAAAAAANHIAAAAA Parliamentary, monetary charges shall evaluate by a observations. Urgent, suitable problems give just at the rises; earlier big others stay always guilty terms. S Books history 1.16 6557.12 2.08562403467702379 +AAAAAAAANJJDAAAA Lovely years help. Possible, good years must imagine even necessar Books history 35.72 11655.58 3.70729188822239413 +AAAAAAAAOCBCAAAA Other, current movements would get in a products. Books history 8.87 18347.84 5.83589992075918761 +AAAAAAAAOPNCAAAA Legal, independent teachers cut. Perhaps common wives might carry already states. Courts rally regions. Besides financial ways could not suffer notably political Books history 3.66 1239.86 0.39436243589177180 +AAAAAAAAPINBAAAA Major, front faces wonder very desirable teachers. Prospective, national plans take industrial, separate locations. Capitalist children save head, economic features. Techniques l Books history 1.92 1668.04 0.53055370571267001 +AAAAAAAAPONBAAAA Trends work to a co Books history 4.91 3816.03 1.21376517206465081 +AAAAAAAAAFFDAAAA Alone sole services keep only; stairs shall eliminate for the woods. Methods must need yet. Other students can Books home repair 2.39 1754.10 0.73033351855711644 +AAAAAAAAAFMCAAAA Alive reforms remember to a rocks. Neighbours could find together with a maps. So anxious circum Books home repair 2.84 819.94 0.34138855550180837 +AAAAAAAAAJFDAAAA Refugees can help as natural publications. Serious, active feet carry alone as well sharp coins. New reasons pay absolutely cautious changes. Practical memb Books home repair 4.33 4572.72 1.90388842538994214 +AAAAAAAAAPKDAAAA Above northern firms can restore either in a tories. Then natural children used to supply publicly chosen things; extra, available circumstances must pay Books home repair 0.40 2992.66 1.24601784826699738 +AAAAAAAABHNBAAAA Here different Books home repair 4.50 3368.22 1.40238524820389416 +AAAAAAAABJMDAAAA Chief Books home repair 4.04 3930.58 1.63652831729675090 +AAAAAAAACALDAAAA Black, relative workers make soft, important cases. Previous p Books home repair 9.53 10606.18 4.41596759469250173 +AAAAAAAACDEEAAAA Taxes disregard earlier for the aims. In part heavy years continue less settings. Breasts accomplish. Weak, appropriate duties mu Books home repair 9.96 6044.52 2.51668408847207200 +AAAAAAAACHACAAAA Members defeat at all new, only bills; original abilities convince; already exciting systems lead shapes. New, real travellers should pursue again short vehicles. Important, only Books home repair 80.60 1171.18 0.48763012956144099 +AAAAAAAACHNDAAAA Professional managers take at least at a applicants. Vulnerable areas must regulate more with a employees. Books home repair 0.38 2026.22 0.84363284987788637 +AAAAAAAACIKAAAAA Completely foreign parties cope with the terms. Children would take terribly visual, total things. Yet good songs will work all right m Books home repair 2.78 1190.62 0.49572412853570149 +AAAAAAAACLLBAAAA Activities bring brief, yellow practitioners. Polish representatives will not prevent for the examples. Annual, ashamed standards use Books home repair 7.44 5309.96 2.21084417661338922 +AAAAAAAADABDAAAA Perhaps european sectors may say practices. Just true years can tell interesting relations. Then private years could not persuade before quickly continuous levels; pale, constitu Books home repair 4.28 61.23 0.02549359862108901 +AAAAAAAADAGBAAAA Chief levels must attack about for a parties. Branches complete really. Just following aims shall not arrive together experienced friends. Actually Books home repair 7.44 7424.19 3.09112069160056914 +AAAAAAAADHHCAAAA States should not hold services. Western manufacturers could not mean even large exercises. Facilities maint Books home repair 7.52 5601.60 2.33227081554617381 +AAAAAAAAEECEAAAA Free, particular nurses get either. Great, evolutionary million Books home repair 0.89 1230.96 0.51252000912323588 +AAAAAAAAFKGBAAAA Military, inc computers ought to maintain entirely even burning sections. Able, outer papers may not cause thus useless, pretty walls. Always im Books home repair 73.73 6564.64 2.73324019683073308 +AAAAAAAAGGDBAAAA Diverse, remaining bits ought to listen along a relationships. Distant stages jail relatively. Short, false applications could appear p Books home repair 1.52 1742.72 0.72559536483658741 +AAAAAAAAGKKCAAAA Houses help general, new attitudes. All central shoes cannot watch. Effects boost to a details. Figures get intently normal, common leaders. Ne Books home repair 1.01 19637.84 8.17637123542653418 +AAAAAAAAGNNAAAAA Even able courses should not vote. Appropriate findings might wait legal things. Sheer, interested levels inform in a meetings. Books home repair 2.99 3714.58 1.54659499536052312 +AAAAAAAAHAAAAAAA Tomorrow different years mean highly in a circumstances. Financial fi Books home repair 0.35 7727.05 3.21721886697837445 +AAAAAAAAHEHBAAAA Open, l Books home repair 6.35 1419.57 0.59104928620838367 +AAAAAAAAHGECAAAA Expenses look away both complete manufacturers. Male advantages use here books. Right rich falls used to say; simple visitors mind records. Conventional profits might arrange Books home repair 7.60 414.17 0.17244298123299750 +AAAAAAAAHGFBAAAA European, local terms bring even everywhere working days; much nice choices grow merely major, black rates; final, other talks can know for example also industrial Books home repair 8.57 772.24 0.32152828024089140 +AAAAAAAAIAIDAAAA Internal exhibitions shall die soon direct movies; services could follow at once social, outer sciences Books home repair 2.25 1729.95 0.72027847353507987 +AAAAAAAAIDEEAAAA However broad boots may not obtain extraordinarily da Books home repair 2.68 2701.11 1.12462868155168622 +AAAAAAAAIDPDAAAA Political, standard statements damage as elegant preferences. Tremendous girl Books home repair 4.06 16118.92 6.71124084085324406 +AAAAAAAAIHHBAAAA British runs wish underneath appropriate pounds. Unable, complex results must not look at the origins. Extra employees find so early thanks. Competent Books home repair 5.60 15.48 0.00644522140542966 +AAAAAAAAIJLDAAAA New, immediate seconds may not give also lines; relevant groups break little golden, political eyebrows. Able cattle doub Books home repair 3.96 1518.63 0.63229370690747035 +AAAAAAAAJJHAAAAA Vast, delicate tem Books home repair 0.83 336.52 0.14011278471286747 +AAAAAAAAKDEAAAAA Corporate stones relieve together early things; forward line Books home repair 8.20 7293.74 3.03680679416269454 +AAAAAAAAKDLBAAAA Words should agree completely level times. Very gentle hours would not interpret. Gr Books home repair 8.23 3906.80 1.62662732472432730 +AAAAAAAAKHGAAAAA However great occupations find very academic homes. Surprised writings suit as free, short shows. Originally possible preparations should accept as yet similar children. Hours re Books home repair 1.86 2705.71 1.12654392822255033 +AAAAAAAAKNGDAAAA Members may not cut probably area Books home repair 0.87 8868.24 3.69236242096172529 +AAAAAAAAKPJAAAAA Similar seats would not see now light soldiers. Rather possible countries take white, proposed boys. Guilty, famous models would not invest often like a fears. Plainly new classes prevent little Books home repair 3.02 3962.44 1.64979348228234450 +AAAAAAAALBABAAAA External hours will not begin never old, empty word Books home repair 1.92 275.50 0.11470662126588312 +AAAAAAAALBCCAAAA Sections will not kick for a systems. Political, lacking arms used to say other authorities. Savi Books home repair 53.64 8876.73 3.69589730014338536 +AAAAAAAALBNDAAAA Planes play sometimes economic, wonderful comments. Responsible, primary costs can bring stra Books home repair 8.00 3496.76 1.45590390191538823 +AAAAAAAALEBAAAAA Of course british lawyers shall describe at least extremely active men. Proposals may gain. Also lexical differences attend bad teams; academic, major contexts could not hold less stead Books home repair 4.97 855.34 0.35612762770802348 +AAAAAAAALJLAAAAA Political, local children will distinguish as necessarily new managers. Directly resulting questions Books home repair 6.97 13643.34 5.68051337271024974 +AAAAAAAALJNBAAAA Issues become at a materials; more complete others should apply seco Books home repair 3.96 2603.64 1.08404627002796343 +AAAAAAAALNICAAAA Real earnings exceed there from a shoulders. Practical days shall not spend now systems. Ages might not sit much. Probably Books home repair 0.86 1450.51 0.60393140185980444 +AAAAAAAALOPAAAAA Scientific contracts transform only variable contacts; just important relations could tell generally on a values. Possible Books home repair 1.94 8305.21 3.45794039202767748 +AAAAAAAALPKBAAAA Extraordinary, economic obligations intend multiple, public patients; again enthusiastic supporters should stop greatly labour, mad trus Books home repair 2.73 1640.87 0.68318930539582445 +AAAAAAAAMBGEAAAA Remarkably political plans would locate separate problems. Sensible areas will not join home social Books home repair 6.39 3591.09 1.49517894940726030 +AAAAAAAAMCFDAAAA Hours might need etc with the holders. Early demands drive usually; at all other responsibilities see so equally italian issues. Simple, senior operations must t Books home repair 6.30 4254.02 1.77119513973681346 +AAAAAAAAMOEEAAAA Spanish, unique colleagues put through a applications. Years will confront normally by no appearances; colleagues will not own still. Sympa Books home repair 2.68 5243.74 2.18327295171238458 +AAAAAAAANCICAAAA British demands can travel easy conditions. Inevitably small pat Books home repair 0.78 3069.27 1.27791503249632335 +AAAAAAAANMFEAAAA Able prices would leave mainly in a matters. Ostensibly necessary schools get far private sales. Laboratories question possibly rare sectors. Likely hands could respond up to good Books home repair 2.22 5893.46 2.45378905323278233 +AAAAAAAAOFKCAAAA Systems cannot show. Global pains sha Books home repair 6.41 748.19 0.31151487101604752 +AAAAAAAAOGJAAAAA Dark, fun calculations must not take away interested feet. High, local films could show normal, visual glasses. Concerned, indian chiefs stick at least. Cultural condition Books home repair 1.87 2172.50 0.90453769401136507 +AAAAAAAAOHGBAAAA Sentences might treat in a persons. Prisoners look best heavy investigations. Western, emotio Books home repair 2.92 1731.95 0.72111118947893383 +AAAAAAAAOOOCAAAA Japane Books home repair 8.75 326.81 0.13606994880545649 +AAAAAAAAOPDCAAAA Democratic, sure places lose in a friends. Other, essential volunteers borrow other, other nurses; foreign hours get indeed enormous designers. Helpful, professional powers lower far from. C Books home repair 4.46 7443.09 3.09898985726998908 +AAAAAAAAPHADAAAA Dutch, quick households ring fortunately small, automatic pubs; objectives st Books home repair 93.40 4131.30 1.72009968942193442 +AAAAAAAAPLACAAAA Industrial, difficult children shall use crops; errors can reach frankly boards. Apparent, special arms may not see always other inter Books home repair 3.19 722.52 0.30082696187668193 +AAAAAAAAPMKDAAAA Suddenly various forms must not involve then local, other economies; continuing, still others cannot know directly only comprehensive products. Odd books go enough southern cases Books home repair 7.64 10446.87 4.34963760618481448 +AAAAAAAAAAICAAAA Rather little years should not reach more new magistrates. Political speakers may lower considerably gates. Kinds would not depend well. Provisions raise. Almost difficult pensions pick yet organi Books mystery 4.25 327.20 0.10733361870342104 +AAAAAAAAAANBAAAA Royal plants find however workers. About genetic peasants come welsh, marine men. So federal eyes develop. Home old services Books mystery 4.32 7859.96 2.57835559188307223 +AAAAAAAAADIAAAAA White changes come much matters. Books mystery 3.16 3490.58 1.14503845591010823 +AAAAAAAAAEMAAAAA Later other operations see; expected, honest animals show respons Books mystery 2.82 18416.84 6.04140000697406092 +AAAAAAAAAENDAAAA Royal advantages succumb again english, new regulat Books mystery 0.58 3081.67 1.01090095583671001 +AAAAAAAAAFEBAAAA Centra Books mystery 1.36 6619.98 2.17159660496416018 +AAAAAAAABEBBAAAA Countries keep much french, addit Books mystery 4.87 25157.14 8.25246599152989476 +AAAAAAAABHODAAAA Always silver months must capture only left mass miles. Characteristics should fall total ways. Courses might work in a spirits; key sources would live again up the records; thoughts can inspect ofte Books mystery 9.69 3901.52 1.27984187054942315 +AAAAAAAACDLBAAAA Primary, single engineers seem new centuries. Close ladies date. Necessary, likely hands cannot retain generally inc prices. Opini Books mystery 1.81 10328.03 3.38797320897766992 +AAAAAAAACGMCAAAA A Books mystery 0.11 6325.20 2.07489793711148765 +AAAAAAAACHLAAAAA Hills may not die reforms. Better Books mystery 5.64 2254.23 0.73947024232827876 +AAAAAAAACHLDAAAA Only present circumstances cannot fall from a players. Sharp relations will blame late eyes. Closest different problems should not write i Books mystery 4.33 9175.56 3.00992071647421134 +AAAAAAAACJMAAAAA Also strategic consultants proceed slightly eyes. Almost stran Books mystery 2.26 23865.71 7.82882951475068011 +AAAAAAAACMLBAAAA Now top documents might mitigate usually ethnic sheets. Big times come partly high records. Social years can seek social, major r Books mystery 2.68 5730.79 1.87990962325604602 +AAAAAAAACNOAAAAA Double workers ought to face with the objects. Satisfactory, other participants help politically urgent, Books mystery 3.56 2094.56 0.68709261733324441 +AAAAAAAADBBCAAAA National specialists go practical chapters. Enough right women stare again for example literary cameras. Most industrial cells shall improve possible, availab Books mystery 3.03 4124.34 1.35293501516891054 +AAAAAAAADEFBAAAA Fortunes could meet emotional meetings. Beautiful women replace beautifully in the things; less previous year Books mystery 5.11 102.48 0.03361720429317417 +AAAAAAAADJEAAAAA Available solicitors emerge. Further true weeks manufacture changes; families save up to right things. Gre Books mystery 3.50 2151.90 0.70590224354490139 +AAAAAAAAECCAAAAA Present, regular adults cannot l Books mystery 7.59 522.99 0.17155993045752497 +AAAAAAAAEDPBAAAA Especially simple sources absorb shortly accessible, new years; glad chapters restrict so southern districts. Modest, particular years could not discard only free men. Now black things could ge Books mystery 3.35 3104.40 1.01835723075458519 +AAAAAAAAEHBEAAAA Days must appear on the police. Direct, late developments should serve always for the papers. Meetings take yesterday women. Medium periods Books mystery 7.03 1997.98 0.65541082975874440 +AAAAAAAAEIDBAAAA Sufficient, whole judges may not show even almost vo Books mystery 75.13 1924.56 0.63132637289687040 +AAAAAAAAEIDCAAAA Words take here free goods. Efficient sales could not ask only. Please local women can talk less than useful permanent colleges. Always free members mus Books mystery 5.23 4082.90 1.33934117299571443 +AAAAAAAAEKFDAAAA Regional, able services should transfer old, social preferences. Other courts might talk a li Books mystery 1.16 954.39 0.31307497663312349 +AAAAAAAAEMAEAAAA Huge, difficult init Books mystery 34.65 621.18 0.20376985717051064 +AAAAAAAAENCDAAAA Difficulties would offer changes. Payable pounds give electric, sure weeks. Tired houses shall not get together most important pools. Bones shall not give foreign, new troops. Books mystery 4.33 12111.11 3.97288894503419799 +AAAAAAAAFMACAAAA Very dead processes may enable drugs. Early units work long police. Easily difficult opportunities ought to seem extra, common eyes. Just quiet subjects must finance ch Books mystery 4.70 475.66 0.15603395193297449 +AAAAAAAAGADCAAAA Also rich lines want noticeably often social difficulties. Animals go; sexual, central cats ought to tolerate. Groups sha Books mystery 3.23 150.35 0.04932032265299313 +AAAAAAAAGDFDAAAA Also significant Books mystery 4.93 1060.69 0.34794528124245618 +AAAAAAAAGGEEAAAA Fine, sure centuries would not form now angry, dead insects; customers cannot pray totally as male laws. Unique procedures reinforce rarely also Books mystery 2.81 5986.79 1.96388702664258571 +AAAAAAAAGGOCAAAA Intermediate, subj Books mystery 9.70 10978.67 3.60140702827227219 +AAAAAAAAGHMBAAAA Hot eyes must invest patently common laws. Whole arts discourage small studies. Policies could need. Reasons hope really independent, international departments. Effective, afraid attitudes Books mystery 0.97 251.85 0.08261605094882821 +AAAAAAAAGPFDAAAA Prices find under way around the languages. Civil, effective products should last really at a hundreds. Main, capable groups will contribute; only indian regulations take now in a feet; total Books mystery 2.73 625.40 0.20515417217946063 +AAAAAAAAHLEDAAAA Advances accept. Lists must not act also old comments. Objectives shall know as to the months; live years can pay possible, inc attempts. Russian years see further pro Books mystery 1.42 15186.66 4.98178231607119854 +AAAAAAAAIABCAAAA Clean systems can realise often s Books mystery 2.73 3145.42 1.03181329750035026 +AAAAAAAAIJCAAAAA Distinguished, huge levels return pretty characters. Months cannot ask right. Overseas studies c Books mystery 6.54 1642.06 0.53865599611289594 +AAAAAAAAIKNBAAAA Voluntary, clear techniques go. Before domestic students ought to live supreme, easy journalists; hands will run overseas such as the skills. Technical, official doctors would Books mystery 5.72 1966.05 0.64493661690666545 +AAAAAAAAIPDAAAAA Good, local rules follow normally high lines. Whole, male activities know again. Books mystery 4.01 5929.90 1.94522501696031914 +AAAAAAAAJABDAAAA Years will appear original Books mystery 4.79 1653.40 0.54237593265353407 +AAAAAAAAJBJDAAAA Problems eat very in a persons; dead ideas happen british things. Short bags should test usually to a others. Also inner visitors expose nevertheless coming, peaceful me Books mystery 4.72 5511.42 1.80794820536188504 +AAAAAAAAJOBEAAAA Expensive rates take as at once white careers. Parts drive all weeks. Therefore other years s Books mystery 0.55 181.72 0.05961083493516403 +AAAAAAAAKFDCAAAA Furthermore little classes say spots. Like days used to provide costs. Friends Books mystery 4.03 13223.74 4.33787245413562633 +AAAAAAAAKIABAAAA Years might give also. Ultimately private stars should make Books mystery 2.78 1284.36 0.42131725708412545 +AAAAAAAAKIKDAAAA Good, low facilities suggest too thereafter asian senses. Far holidays defend delicate members. Cautious reports treat on a talks Books mystery 0.25 5386.71 1.76703874451682502 +AAAAAAAAKNLDAAAA Strange, necessary weeks hope all. Dead sons know too. Heavy, social waters used to move pupils. Heels provide. Eastern trees used to allow currently bad children. Articles would not clear Books mystery 4.09 5477.40 1.79678839573997066 +AAAAAAAALFFBAAAA Bitter, nice students like general books; maximum, holy members draw indeed sure, strong lines; forests must not adapt opposite, r Books mystery 6.38 2322.45 0.76184890818386367 +AAAAAAAALPOAAAAA Everyday, low cases could contribute again through a developments. Useful, unable answers might not assign local da Books mystery 1.87 8562.04 2.80866362067065732 +AAAAAAAAMJLAAAAA Free, peaceful years should not help ahead animals. Then helpful others Books mystery 27.03 92.46 0.03033027623874789 +AAAAAAAANIHDAAAA Howev Books mystery 3.41 6376.36 2.09168029631951644 +AAAAAAAANPECAAAA Sorry theories decide there wages. Books mystery 2.59 4969.90 1.63030975426079530 +AAAAAAAAOEEDAAAA Other courses discuss full leaves. Connections excuse; objective, international sessions go. All expensive surve Books mystery 3.01 1617.54 0.53061253544477894 +AAAAAAAAOEHBAAAA Banks will employ of course real, dead resources. Sisters shall not go short effects. Hopes run c Books mystery 3.63 4915.26 1.61238582722548074 +AAAAAAAAOEMCAAAA Seconds preve Books mystery 4.51 2037.80 0.66847325242613507 +AAAAAAAAOGLDAAAA Right developments would not seek variables; numbers like impatiently Books mystery 3.84 11928.22 3.91289430712261892 +AAAAAAAAOIAAAAAA Limits ought to eat less; actual costs would smash more main rules; magnetic, constitutional expressions can head years. Quickly western children may not wonder also useless, other millions; comm Books mystery 10.39 6043.00 1.98232597134710679 +AAAAAAAAOOGDAAAA British, quiet residents trace particularly. Years should reduce now libraries. Special, general figures gain Books mystery 2.22 6385.64 2.09472447719227850 +AAAAAAAAPNKCAAAA Most small ministers appear agencies. Industries review so much as solicitors. Far from distant children hear still terms. Particular, available days learn already long-t Books mystery 3.79 3704.73 1.21528752206334055 +AAAAAAAAPPEDAAAA Sizes could not continue home; obligations will not lack notably current buildings. Measures burn there then useful thousands. Historic, Books mystery 7.35 5443.06 1.78552361436382311 +AAAAAAAAAAABAAAA Inches c Books parenting 0.16 4582.16 1.47127016656624148 +AAAAAAAAAAFCAAAA Certain signs prepare societies. Economic reasons can i Books parenting 0.98 1989.28 0.63873114796229133 +AAAAAAAAADLCAAAA Golden dogs could hear only available feet. Big, serious patterns used to use here with a days; otherwise long reasons should not trave Books parenting 1.58 566.43 0.18187308178852684 +AAAAAAAABCEDAAAA Luckily economic c Books parenting 9.18 122.92 0.03946796464425564 +AAAAAAAABEMCAAAA Men become most so living studies; private nurses come frequently in a feet. Points will Books parenting 1.38 4878.48 1.56641454732922415 +AAAAAAAACCOCAAAA Other changes mean. Miles form. Local, illegal authorities take again inside the figures. Players would love properly Books parenting 14.38 2483.90 0.79754700113786669 +AAAAAAAACPIAAAAA Popular circumstances should not take relations. Secret questions should get after the players. Automatic methods cope please in a effects; unli Books parenting 5.60 9646.64 3.09740682115084758 +AAAAAAAADLKBAAAA Original, able troops reduce jointly. Crowds move american feet. Cities move. Legs transfer loudly so central germans. Households could c Books parenting 4.02 877.39 0.28171817034838474 +AAAAAAAAEFBAAAAA Typical, right programmes tell against a reforms. Outside friends can inhibit again either military stairs. International men must launch legall Books parenting 65.75 4078.44 1.30953242534752647 +AAAAAAAAEKGDAAAA Favorite, small son Books parenting 1.77 4476.61 1.43737947613180297 +AAAAAAAAELEBAAAA Improved loans read years. Now constant tears perform now local negotiations. Specifically concerned problems ought to know more than previous steep plants. Cont Books parenting 0.48 5231.60 1.67979664686696862 +AAAAAAAAGCJDAAAA So plain prisoners make improvements. Contemporary roots will resume in the computers. Firms accept modern, present names. Essential, collective sons cannot examine in the d Books parenting 5.38 18382.40 5.90234228178136019 +AAAAAAAAGEOCAAAA Soft friends could make clean, brave feet. Rapid standards should not spread problems. Careers use quantities; british, other visitors should pursue wide, sudden sh Books parenting 4.17 7509.00 2.41103926548743546 +AAAAAAAAHCDBAAAA Crazy years could cry even clergy. Other, philosophical sides might take years. Already senior hours cannot believe early strengths. Fields will not find little jewish councils. Events might not o Books parenting 1.37 8851.94 2.84223930160325602 +AAAAAAAAHIEDAAAA Prime, flexible records say upwards at least easy schools. Here good investors can spend more at a cus Books parenting 7.33 6260.65 2.01021081069035995 +AAAAAAAAHMOBAAAA Arms shall get thus famous, clear conditions. Royal languages might not understand in a films. Scientific, notable views would achieve like a years. Large, nervous students Books parenting 2.05 2365.43 0.75950787185536616 +AAAAAAAAIBOAAAAA Main contents set within a communities; rules date at Books parenting 1.39 1973.40 0.63363229278371356 +AAAAAAAAICADAAAA Leaders restructure so. Years used to go from a years. Shoulders supply thus original tracks. Securely necessary Books parenting 2.01 2314.86 0.74327052258706151 +AAAAAAAAICBDAAAA Faces may occur existing houses. Ruling, annual arguments allow all but for a elections. Future, spanish subjects take. Then prim Books parenting 8.01 13033.96 4.18502987678687100 +AAAAAAAAIIKDAAAA High fields shall join then. Diffi Books parenting 1.11 3833.50 1.23088547399734770 +AAAAAAAAINOCAAAA Narrow, Books parenting 7.17 950.12 0.30507079863163167 +AAAAAAAAJFNBAAAA Very strong arrangements should not cover parliamentary, fundamental implications. Parents renew then; major, basic structures settle; only long-te Books parenting 7.59 3460.43 1.11109769682656629 +AAAAAAAAJLLAAAAA Pretty eastern facts should not join. Too labour things mean in particular. Closer intensive problems Books parenting 1.18 11548.91 3.70820022420834975 +AAAAAAAAJNFBAAAA New friends must not gather by a blocks. Empty opportunities ought to remind else single families. Early years should not use suddenly abou Books parenting 4.28 11681.79 3.75086621137015165 +AAAAAAAAKBJDAAAA Source Books parenting 8.78 5480.98 1.75986922271292103 +AAAAAAAAKDPAAAAA Good countries need once again. Most economic patients appear there only real trees. Apparently jewish policies Books parenting 9.76 3680.94 1.18190050258400862 +AAAAAAAAKGNCAAAA Small, true kids can go genuine objectives. Scottish games give ever. Scientific, similar trees remark. Boot Books parenting 8.58 10853.90 3.48504182763005404 +AAAAAAAAKHDDAAAA Widespread lands get curious, certain reasons; issues ought to accept sales. Easy, other others might bomb large payments. Econo Books parenting 4.78 8024.99 2.57671673926541680 +AAAAAAAAKHJAAAAA Forces can measure now groups. Resources form rat Books parenting 4.43 6742.48 2.16491996627563242 +AAAAAAAAKNFAAAAA Equal voices build. High streets would harm simply individual, black methods. Substantial rooms land as current savings. Again very opportunit Books parenting 7.81 26.70 0.00857301217053063 +AAAAAAAALEICAAAA Overall, high heads cannot see explicit, bad bodies; opportunities can accommodate little leaders. Light times u Books parenting 6.61 13341.53 4.28378648177900984 +AAAAAAAAMABDAAAA Meanwhile thorough roads put also more other trees. Never religious costs want just especially direct nights. Young, excellent aud Books parenting 2.67 3546.05 1.13858913135993082 +AAAAAAAAMAFCAAAA Common circles may win children. Tiny things must need as beside a words. Permanent yards remain fully. Slight, general ways avoid new, possible arts; therefore educational conditions ou Books parenting 4.26 9853.55 3.16384284917348778 +AAAAAAAAMCFBAAAA Sites will not manage most generally immense woods. Fine employers avoid in a men; reasons ought to think here; only corresponding areas Books parenting 58.45 12923.27 4.14948880123795580 +AAAAAAAAMGCDAAAA Records face long lips. Main researchers will know unequivocally ameri Books parenting 1.24 16478.74 5.29110256835243338 +AAAAAAAAMLACAAAA Corners would not descend often plain new activities. Just local trusts think Books parenting 8.15 9940.76 3.19184481139790637 +AAAAAAAAMMJBAAAA Open, large roads might tell friends. Used, old arms will drop as good as natural others. Sad programmes participate Books parenting 4.27 2597.90 0.83415087332664917 +AAAAAAAAMNKDAAAA Days could meet just. Folk might alter possibly tories; serious, basic things wait suffici Books parenting 5.54 8776.83 2.81812248721641872 +AAAAAAAAMPBEAAAA Stations may not reme Books parenting 0.88 3316.92 1.06501855912645951 +AAAAAAAAMPNAAAAA Economic, free bits post quite issues. Perhaps back sales used to affect d Books parenting 0.09 19263.00 6.18509114010979749 +AAAAAAAANDECAAAA Genuine cities say. Practices prove together elsewhere simple Books parenting 1.52 1712.57 0.54988327538897554 +AAAAAAAANPFEAAAA Se Books parenting 3.22 2194.90 0.70475297427332163 +AAAAAAAAOCAEAAAA Partners will not locate. General, other losses cannot restrict else protective kilometres; children carry unusual, long groups. Yet true reservations differ never long-term Books parenting 1.02 6482.66 2.08149524634502309 +AAAAAAAAOFDDAAAA Profits could not cling through a terms. Later democratic arms might not work all. Sometimes apparent arti Books parenting 6.57 0.00 0.00000000000000000 +AAAAAAAAOFICAAAA Else emotional lives declare also c Books parenting 7.67 4780.68 1.53501227803042655 +AAAAAAAAOKJCAAAA Previous floors keep complex computers. Books parenting 9.60 5787.26 1.85821162599344996 +AAAAAAAAOKKBAAAA Lists used to miss little names. Prime roads should not help from the minutes; in order various exceptions help Books parenting 1.19 4186.16 1.34411987369994445 +AAAAAAAAONDEAAAA Theories look. Just young regions Books parenting 45.83 1849.39 0.59381434374747746 +AAAAAAAAPCGCAAAA Foreign, simple stocks may draw still; Books parenting 2.55 18500.06 5.94012133091936148 +AAAAAAAAAKICAAAA Careful, long customers may think about just professional meetings. Students could not drink. British, basic commentators remember espec Books reference 1.77 6207.69 2.15509748883540916 +AAAAAAAAALADAAAA Bills emerge later in a yards. Ev Books reference 2.72 1496.80 0.51963772696266090 +AAAAAAAACEDCAAAA Examples will talk there estimated, short initiatives. Benefits ought to prove too negative Books reference 0.17 6141.90 2.13225745272044827 +AAAAAAAACGFDAAAA Sorry services must not recall much main details. Sexual, major secrets will not go results. P Books reference 7.54 1423.78 0.49428768231887850 +AAAAAAAACGMDAAAA Flexible, previous patterns must not manipulate essential, dull criteria. Much possible players will include firmly working, important duties. Far english busi Books reference 6.38 13587.29 4.71704201709145697 +AAAAAAAACLJBAAAA Funds shall call more able countries. Books reference 0.39 913.90 0.31727479868464444 +AAAAAAAACNAEAAAA Indivi Books reference 3.76 2162.13 0.75061752979541556 +AAAAAAAACOOBAAAA Hitherto certain kinds evade also by a months. Poor points might make even just selective passengers. Old, general qualities could overcome over; recent variables might s Books reference 56.16 1298.61 0.45083294268504882 +AAAAAAAADAJBAAAA Difficult, rapid sizes say so; initial banks stress high single sports; prisoners used to think likely firms. Good, current services must take human, precise persons. Signals m Books reference 7.77 9585.22 3.32766029745927077 +AAAAAAAAEDAEAAAA Royal, educational days can add black, long-term matters. Different executives should not remai Books reference 4.86 9194.30 3.19194625401709854 +AAAAAAAAEEDCAAAA Classical, labour books make in addition finally significant suggestions. Ethical figures could sell as the levels. Regardless plain scholars set in a companie Books reference 80.47 2466.20 0.85618022597228374 +AAAAAAAAEHNCAAAA Cruelly shared examples shall not investigate then in vit Books reference 0.28 610.19 0.21183708218555990 +AAAAAAAAEJABAAAA Male, small legs allocate today to a programs. Video-taped circumstances afford short, royal changes. Planned, appropriate names can enter usual periods. Very consta Books reference 4.40 9663.14 3.35471145438399721 +AAAAAAAAELGBAAAA Often other ideas must not understand possible, static groups. Late Books reference 8.13 705.22 0.24482824546272563 +AAAAAAAAENCEAAAA Possible solutio Books reference 2.63 10773.86 3.74031542023913264 +AAAAAAAAFGKBAAAA Still short documents ought to give more longer individual parties. Brief, expensive reforms should give now. As perfect sect Books reference 1.16 4401.20 1.52794599405936875 +AAAAAAAAGLODAAAA Great speeches would draw too particular, full things. Available, real lives shall like long, supreme skills. Grim men would n Books reference 4.95 7141.72 2.47936073450278901 +AAAAAAAAGPGBAAAA Ever only sides should not ensure clearly familiar, running points. Persons bear free, huge products. Organizations blame. Recent, parliamentary communities complain both perfect, l Books reference 5.85 4618.08 1.60323930660858167 +AAAAAAAAHJBCAAAA Dead, blue homes should write more small objectives. Systems could underpin all so blue exchanges. Better adult arts make very governments. Quick managers talk and Books reference 2.83 3913.25 1.35854645579678832 +AAAAAAAAHKEBAAAA Damp, happy roads Books reference 4.29 12407.36 4.30741070818241603 +AAAAAAAAIEPCAAAA Italian pati Books reference 4.42 7902.99 2.74364762146488472 +AAAAAAAAIFNDAAAA Classes used t Books reference 1.61 7530.59 2.61436308811313771 +AAAAAAAAIIECAAAA Dangerous parents would not advise almost previous, important matters. Books reference 7.62 1064.34 0.36950241736734266 +AAAAAAAAIMACAAAA Utterly free reasons control powers. Resources think too systematic sy Books reference 5.69 6131.92 2.12879273831966837 +AAAAAAAAINEAAAAA Therefore secondary countries get eventually prospective lives. Directly complete wings see as g Books reference 6.19 4028.40 1.39852259439897325 +AAAAAAAAJEFEAAAA At present pink police would not endorse yet bright rules. Photographs shall te Books reference 5.24 7033.41 2.44175920977849331 +AAAAAAAAJOGCAAAA Equal, strong requirements use broadly remote pictures. Books reference 96.89 15194.39 5.27497212866393982 +AAAAAAAAKAMAAAAA Relative, possible papers may change only current, tropical services; following procedures bring ever delicious questions; never convenient women may want secondary ch Books reference 3.67 2.16 0.00074987806670186 +AAAAAAAAKELAAAAA Eyes alleviate yet; major women get that blue scientists. Wild interests suffer forthwith years. Women might complete in a commitments. Japanese, victorian Books reference 8.24 12242.59 4.25020820399238554 +AAAAAAAAKFHAAAAA Clear points create however from a bases. Social, wrong rates contribute. More whole legs find now now unha Books reference 0.65 9377.23 3.25545328861977061 +AAAAAAAAKGCCAAAA Glad, certain others ought to protect narrow, american friends; thi Books reference 9.25 2557.68 0.88793895076019410 +AAAAAAAAKMJBAAAA Long son Books reference 6.53 13751.99 4.77422021967747397 +AAAAAAAAKNPDAAAA Historical arguments can point much big times. Lines bri Books reference 7.40 4482.72 1.55624694776193163 +AAAAAAAALDIAAAAA Types shall serve quite possible emotions; hard weekends appear months. There difficult colours form probably. Rules know however green manufac Books reference 4.01 2684.41 0.93193526899775290 +AAAAAAAALHBAAAAA Also real addresses give in a advantages. Perfect, interested humans could fall never at a years. Sophisticated interp Books reference 8.60 936.71 0.32519364993532475 +AAAAAAAAMAMBAAAA Much political attitudes must not understand more. Holy years shall not link large friends. Now occasional supporters may write also. Southern difficulties used Books reference 3.32 7569.18 2.62776021524000108 +AAAAAAAAMDGCAAAA Actions cannot go perhaps publications; huge, willing girls wo Books reference 9.60 2251.62 0.78168539469779966 +AAAAAAAAMHHAAAAA Successful solutions find clearly as socialist problems; individual systems Books reference 9.20 2974.66 1.03270013421081565 +AAAAAAAAMKOCAAAA Too nuclear windows ought to contemplate for example active, constitutional appeals. Again short partners clear to the issues. There political sheets end s Books reference 3.51 295.80 0.10269163524556059 +AAAAAAAAMLJDAAAA Cities regard only. Operations used to make later; personal, written years used to interfere for a agreements. Obvious, sufficient protests tell. Issues pay effective own Books reference 2.70 445.16 0.15454431489490789 +AAAAAAAAMPPBAAAA Here special fruits sti Books reference 2.31 6938.36 2.40876110318589515 +AAAAAAAANCABAAAA Years decide pot Books reference 4.03 15341.75 5.32613047677004465 +AAAAAAAANINDAAAA Structures drop home revolutionary, formal hands. Ears Books reference 3.42 1450.10 0.50342508542794934 +AAAAAAAAOAFDAAAA Predominantly on Books reference 8.46 11177.59 3.88047665721577287 +AAAAAAAAOIPBAAAA Really different purposes answ Books reference 81.85 4832.22 1.67758138494355241 +AAAAAAAAOKBDAAAA Kinds play sooner; old causes would publish. Great, Books reference 2.90 463.44 0.16089050520014402 +AAAAAAAAOMPAAAAA Relations preclude most primary records. Hardly common f Books reference 3.01 45.64 0.01584464581679305 +AAAAAAAAPDEAAAAA Particularly natural children put hardly. Parties weep into a days. Heavy hands will not take mad, lonely children. Ye Books reference 4.55 1000.50 0.34733935450704318 +AAAAAAAAPEKCAAAA Little, num Books reference 4.79 11088.98 3.84971429819241545 +AAAAAAAAPFFAAAAA Democratic, fresh operations shall not explain fully decisions; contra Books reference 1.68 140.25 0.04868999946987787 +AAAAAAAAPOIDAAAA As progressive minutes apply as firms. Involved, Books reference 4.35 18398.21 6.38722877109947712 +AAAAAAAAAAGCAAAA Both gross guns ought t Books romance 22.07 2932.20 1.53691964340235494 +AAAAAAAAAAJCAAAA Matters care too expressions; economic Books romance 5.87 4968.70 2.60435598941862117 +AAAAAAAAACNCAAAA Internal, additional structures pretend trains. Useful payments should make fingers. Books romance 0.64 4689.33 2.45792353570560163 +AAAAAAAAADEEAAAA Following, very poli Books romance 1.59 7979.33 4.18238490491430082 +AAAAAAAAAGBDAAAA Likely weapons see. Items improve half. Short, human resources depend white, local texts; fully permanent way Books romance 6.42 22088.52 11.57775059057560371 +AAAAAAAAALIAAAAA Full days keep full, visible bottles. Big, domestic countr Books romance 4.62 11680.82 6.12252974184813303 +AAAAAAAAANADAAAA Teachers arise clear often old services. Other minutes could cost by a attempts; open conscious goods detect yet disastrous stones; thus slight men tell for a countries. Capitalist bodies wou Books romance 0.25 4832.22 2.53281967097801228 +AAAAAAAABCBDAAAA New, small beds will come instead in a stories. Female, other systems could not Books romance 4.36 9867.04 5.17183261654620160 +AAAAAAAACFGAAAAA Part-time architects buy. Silently national skills understand free parts. Only european millions shall not attend at all other informal words. Empty, redundant holes contain again acceptable relatio Books romance 1.12 1104.46 0.57890535071010332 +AAAAAAAACFJAAAAA Similar consumers will live once on a eyes. More likely teams pass particularly. Just other workshops Books romance 3.59 1239.88 0.64988606761534406 +AAAAAAAACGKCAAAA Future years can reform as before social suppliers; particular, judicial individuals resume vaguely remaining aff Books romance 0.52 6031.54 3.16144611757964666 +AAAAAAAACHJCAAAA Crucial, different affairs could not forgo; public p Books romance 5.62 4775.42 2.50304781512054902 +AAAAAAAACIDBAAAA For example new resources find perhaps necessary opportunities. Main systems move spontaneously necessary m Books romance 6.68 3560.08 1.86602444720136955 +AAAAAAAACIJDAAAA Rather aware thanks may not work with a chi Books romance 2.35 2220.62 1.16394328440493058 +AAAAAAAAEGDDAAAA Islands meet only for Books romance 6.79 2450.58 1.28447736843630822 +AAAAAAAAEIKBAAAA Minutes will defend. Now new courses could know definitely international forces. There capital accounts should not lift more pro Books romance 72.49 1876.47 0.98355623874743093 +AAAAAAAAFLJDAAAA More simple principl Books romance 6.44 6567.15 3.44218738018203917 +AAAAAAAAFOECAAAA Late, dark looks would not make citizens. Safe, great curtains use as by the children. Signs would prove neither romantic moveme Books romance 4.68 2862.64 1.50045960302479959 +AAAAAAAAGBGBAAAA Problems inherit. Sure edges must become enough revolutionary years. Systems burst however slowly strong issues; cultural site Books romance 1.60 775.70 0.40658501036327902 +AAAAAAAAGDNDAAAA Possible, common bars cannot rid mainly ultimate years. Drugs could bring of course large, good rules. S Books romance 3.33 273.51 0.14336092069673900 +AAAAAAAAGFLAAAAA Standard, geographical scales may hope equal, sure problems. Strong associati Books romance 7.58 4049.00 2.12229303462797052 +AAAAAAAAGKDDAAAA Probably just results receive perfectly on the countries. Bold girls will pass religious years. Here public conditions ought to consider most sources. Different, able years go rarely ita Books romance 5.44 1710.73 0.89668322132109361 +AAAAAAAAGLMDAAAA Even sure children build there imaginative novels. Real, quick members shall not exercise unlikely, vast times. Open regulations buy all catholic days. Domestic, palest Books romance 6.42 49.14 0.02575684853584057 +AAAAAAAAGOPDAAAA Silver, political interviews might know in common families. Far possible houses shall insist in a places. Whole, political gardens would adopt eggs. Others might live even offi Books romance 6.13 5432.94 2.84768849581419762 +AAAAAAAAHHLAAAAA Cultural, harsh conditions describe Books romance 4.72 1495.08 0.78364975801718601 +AAAAAAAAIAACAAAA Distinctive hours work more federal, proper plants; crimes may ensure therefore; players work increasingly previous, genuine needs. Hostile, young schools will offer very new, implicit changes; Books romance 47.76 1911.06 1.00168666998175583 +AAAAAAAAIBFAAAAA Particular bombs could illustrate suddenly planes. Western months expect just special, relevant readers. Able demands ought to achieve for a cars. Suitable counties must stud Books romance 0.88 1663.75 0.87205854195166361 +AAAAAAAAICDAAAAA Levels tear only. Colleagues may not see hot forests. So effective residents must help completely in a hands. However professional classes ought to seem very; political Books romance 4.81 1069.40 0.56052856785160575 +AAAAAAAAIHBAAAAA So only things know prac Books romance 2.71 3443.44 1.80488731221519852 +AAAAAAAAIHDEAAAA Ways used to contain only double cigarettes. Intensely increased feelings Books romance 76.83 18974.38 9.94546666099883214 +AAAAAAAAIJFDAAAA Views balance quite other degrees. Slow passages promote due major animals. Sons would say. Possible, other schemes cannot restart either important, new Books romance 3.75 745.80 0.39091285384676227 +AAAAAAAAIKODAAAA Premier, good budgets could put high, slow members; traditions could not join however. Students laugh for a effects. Carefu Books romance 9.00 1184.75 0.62098954625228157 +AAAAAAAAILNCAAAA Contacts remove basically blue, labour details. Full measures hold then families. G Books romance 66.85 845.81 0.44333333455635558 +AAAAAAAAIMDCAAAA Subject children would not like sufficiently great levels. Yet busy hotels must not help behind Books romance 9.33 1361.15 0.71345002817581182 +AAAAAAAAJDMBAAAA Large thoughts make Books romance 0.85 2228.59 1.16812077896802885 +AAAAAAAAJGBAAAAA Specially clinical muscles can pass causal, following changes. Dishes could use at present areas; even c Books romance 5.00 276.00 0.14466606015246230 +AAAAAAAAJJPBAAAA Teachers play apparent indians. Professional corners accept consequences; extensively necessary men will not know only economic clean stairs. Divisions could Books romance 0.78 379.40 0.19886341747044999 +AAAAAAAAJLBBAAAA Stages choose physically to a families Books romance 6.13 1969.70 1.03242296624023550 +AAAAAAAAKBEBAAAA Illegal technologies might distinguish that on a change Books romance 2.73 1019.24 0.53423708387607130 +AAAAAAAAKBLBAAAA As single women would get ideas. Rural classes may hear quite available, high sequen Books romance 1.38 894.27 0.46873375946573356 +AAAAAAAALCADAAAA Senior fans cook frequently. Fin Books romance 4.36 5607.44 2.93915308819320006 +AAAAAAAALMLAAAAA Mammals take at all. Profound weeks must know parts. Too low earnings can share directly new gaps. Equal block Books romance 4.99 179.00 0.09382327814235780 +AAAAAAAAMABAAAAA Fine, real rows could think short, united others. Twice moving molecules list enough really vague assessments. Days put with a lines. Importa Books romance 4.85 950.33 0.49811774255322283 +AAAAAAAAMAOAAAAA Associated words produce simply. Frantically tough forms take there across right years. Recent fears appear also fierce examples. Incredibly coastal te Books romance 2.28 99.82 0.05232089175514053 +AAAAAAAAMDNBAAAA Historical, new notes should say levels; largely low prisons present at once enough useful winners. Yet worthwhile sons give different, social beaches. Minutes want guns. Industrial Books romance 65.28 3120.61 1.63567519555208473 +AAAAAAAAMHDAAAAA Complete, foreign makers prevent conservative gardens; full prisoners would look so good goods. Then only cir Books romance 3.56 510.48 0.26756931299503245 +AAAAAAAAMLEEAAAA Local, strong letters should not make also ba Books romance 6.39 3270.83 1.71441336785680534 +AAAAAAAANDMDAAAA At all chemical branches make as existing things. Directly civil students must not afford much beautiful companies. Past police offer well perhaps chan Books romance 36.28 3753.37 1.96733786302336027 +AAAAAAAANIKAAAAA Minor democrats can wonder impatiently real backs. Early, Books romance 2.77 1091.04 0.57187122561138576 +AAAAAAAANMGDAAAA Surely local universities may know perhaps primitive computers. About bad sides will provide carefully about a workshops. National, sheer references ought to develop already also long-t Books romance 5.58 112.88 0.05916632199278965 +AAAAAAAANNDCAAAA Financial things will die only pai Books romance 1.33 1782.43 0.93426494781722240 +AAAAAAAAODHCAAAA Debts should not go into a eyes. Legal troops pursue wholly friends. Inc families will meet never; potatoes should give all various users. New women st Books romance 4.80 6935.94 3.63548954077488907 +AAAAAAAAPDEDAAAA Also genuine men identify. Gradual, useful things used to see below patterns; superb, hidden ways would fail even huge yea Books romance 2.08 1555.12 0.81511986762426513 +AAAAAAAAPENCAAAA Gains keep still. Possible, final clothes kill perhaps in the conclusions. Methods would proceed for a hopes. Other, particular ways find perhaps in a demands. Adverse, other men admit eviden Books romance 1.93 3352.42 1.75717896150839737 +AAAAAAAAPLHBAAAA Racial minutes used to come enough teenag Books romance 3.47 4982.66 2.61167315680894137 +AAAAAAAAACCAAAAA Then modern features should improve otherwise available qualifications. Personal purposes go with a years. Ministers remove big arts. Linear, poli Books science 4.66 527.85 0.17402980157734269 +AAAAAAAAAEJDAAAA Organizations make enough horrible requirements. Grateful, only funds reassure anxiously yesterday great years. Extra Books science 5.13 36276.15 11.96008560479287668 +AAAAAAAAAGIDAAAA Ac Books science 1.13 11382.07 3.75261794759766011 +AAAAAAAAAIBBAAAA P Books science 7.15 115.77 0.03816885503193893 +AAAAAAAAAMOAAAAA Confident views gain to the resources. Jobs could direct kings. Attitudes might not support as aware jobs. Happy accounts cannot test. Professional, joint interests will support in Books science 78.67 7479.68 2.46601728949894583 +AAAAAAAAAPLDAAAA Continuous members shall look usually about careful supplies. More than negative sports become probably other leaves. L Books science 47.51 97.92 0.03228378927811575 +AAAAAAAABEGCAAAA Obvious relationships put originally. Pounds give well central, british leaves. Differences ought to ask also central states. Tests grant for a chapters. Soon active heads should want Books science 4.26 2414.14 0.79593124027645368 +AAAAAAAABEHBAAAA Gently independent fears call now statutory sciences. Friendly, quiet needs stumble too. So famous cattle teach too only services; public forces collect pure friends. Arms might make im Books science 4.68 5668.22 1.86878696958743084 +AAAAAAAACAECAAAA Later other words comfort historic, social birds. Large, english interests muster there ag Books science 1.74 2463.16 0.81209291664913785 +AAAAAAAACAOAAAAA Ways create things. Popular opportunities regard eyes. Intact conditions show years. Variable banks could run legally. Sexual, mechanical dates shall not carry however fingers. Forms Books science 2.88 10151.52 3.34691107570034261 +AAAAAAAACDKBAAAA Now educational levels lift perhaps men. Types use not. Very environments might go for sure at once common p Books science 71.85 6430.06 2.11996223535172516 +AAAAAAAADCEEAAAA Little, able companies could not combine particles. Private kids participate in common; unable, only detectives introduce; very good skills go. Copies miss. Strategic m Books science 1.07 7269.76 2.39680759745174345 +AAAAAAAADNCBAAAA Regular teachers serve together events. Other arms would not use. Dou Books science 3.59 8847.06 2.91683640493103230 +AAAAAAAAEEEBAAAA Aware parts hang experienced, new groups. Handsome, perfect forms will grasp tonight in terms of the tears. Effective, economic subjects deny in the o Books science 3.18 38.60 0.01272624863291736 +AAAAAAAAENIAAAAA Just essential errors permit never too bad applications. Ideas could buy men. Anxious wives would not pull royal, common towns. Adults Books science 3.22 10051.00 3.31377007796508735 +AAAAAAAAFCPAAAAA Domestic copies cannot get additional victims. Pieces should not determine now british, gold depths. Local, available stocks punc Books science 3.99 3769.53 1.24279730593888526 +AAAAAAAAFPOAAAAA Complaints can involve very vital adults. A little practical initiatives remain traditionally important months. Clear new transactions create perhaps new, personal princip Books science 1.15 3928.72 1.29528154220505402 +AAAAAAAAGCCDAAAA Distinguished, assis Books science 6.29 16.68 0.00549932194811040 +AAAAAAAAGCCEAAAA Old prices help general trials. National, prime men ought to compete about a posts. Suspicious, extreme mistakes might not make gently other characters. Acc Books science 1.53 3227.96 1.06424408127232946 +AAAAAAAAGEHDAAAA Spanish ranks can deal all but conservatives. Local metres shall not go no longer with a processes Books science 2.91 4385.32 1.44582053510116972 +AAAAAAAAGGBAAAAA Particular ears ought to know streets; tears could pr Books science 1.38 4417.02 1.45627188436706299 +AAAAAAAAGIAAAAAA Useful examples might understand evidently. Royal shops ought to leave in order. Also huge experts stay continuous, long organisers. Often burning services flee global circumstances. Fine, ex Books science 2.78 7923.96 2.61249443309046200 +AAAAAAAAGJGBAAAA Accounts accept Books science 1.24 4454.22 1.46853655921536677 +AAAAAAAAGKEDAAAA Small years turn as beside a problems. Famous, significant attitudes defend again subtle machines. Pp. double less. Human men appear in a regions. Exclusively warm Books science 1.75 3606.79 1.18914265043316062 +AAAAAAAAHFDEAAAA Certain, long councillors smile then fresh eyes. Lights attend initially after a preferences; national genes admit. Wide single plans improve never Books science 2.09 2209.49 0.72845904383276100 +AAAAAAAAHGDAAAAA Problems could not find small, late years. Demands might get only normal, available communications. Quiet mothers leave women. Fair interes Books science 0.21 8916.11 2.93960188337929509 +AAAAAAAAHJPDAAAA Marks remember Books science 1.41 1407.04 0.46389484135906840 +AAAAAAAAHMDDAAAA Things prejudice unfortunately. Available lives used to get for an readers. Roughly good articles might express open years. Black m Books science 9.38 11566.26 3.81334457287478571 +AAAAAAAAHNIDAAAA Small, stupid members lack hands. Literary terms would understand sure ordinary acids. Lovely, Books science 0.22 2581.68 0.85116843447228203 +AAAAAAAAIHEAAAAA Conditions must like most still desperate concessions. Parts shall not raise sometimes places. Local, prof Books science 4.37 214.32 0.07066035251313079 +AAAAAAAAIJHBAAAA Male, major regulations could get. Books may not bring. Upper, musical girls take well special, curious parents. Criminal, equal knees stop just a Books science 3.41 7411.80 2.44363755485639582 +AAAAAAAAILGAAAAA Courts receive high male limitations. Political, little parents may establish tomorrow unique minu Books science 9.26 10412.18 3.43284952048418299 +AAAAAAAAIMADAAAA Local, contemporary tanks provoke yet. Well red quantities should spend only deaf new firms. Books science 2.13 6975.01 2.29962983101256232 +AAAAAAAAIMMAAAAA Young officers depend very well unnecessary players. Personnel will consider apart types. Most universal courses could enable arrangements. Magic, equal responsibilities detect; value Books science 5.89 6948.34 2.29083685041567357 +AAAAAAAAIOHAAAAA Pounds realise fairly formal, casual residents. Good areas shall stick etc disputes. So small police find variable, certain programs. Results think children; dogs will take prices. Old, traditi Books science 44.25 3791.67 1.25009676564698863 +AAAAAAAAIOOBAAAA Left times used to tell trees. Right t Books science 7.96 2094.92 0.69068582347334800 +AAAAAAAAIPCBAAAA So clear employees could tell experiments. Hands would control demands; well ethnic sites afford then bottom programmes; times flow easily premises. Alter Books science 1.28 10461.12 3.44898482121203209 +AAAAAAAAJLLDAAAA However major deb Books science 0.66 2219.28 0.73168676336945170 +AAAAAAAAJNDDAAAA Thereafter strange rates shall not inhibit now on a heroes; eyes may not provide. Books science 8.37 11495.90 3.79014719324234879 +AAAAAAAALAPCAAAA Due proposed concepts afford indeed yesterda Books science 1.34 10405.19 3.43054494851671946 +AAAAAAAALKJBAAAA Earnings feel possibilities. Single, poor problems make full, sho Books science 2.75 17541.34 5.78330192213830518 +AAAAAAAALNGBAAAA Direct schemes rival pa Books science 78.33 9776.79 3.22336425833730836 +AAAAAAAAMBLCAAAA M Books science 42.63 5228.32 1.72375389255063431 +AAAAAAAAMCPCAAAA Clear spirits shall not co Books science 2.11 1098.47 0.36216068227463034 +AAAAAAAAMLBEAAAA New, political bish Books science 1.33 1836.00 0.60532104896467022 +AAAAAAAANKOAAAAA Professionally uncomfortable groups would not protect again there dependent users. Standard fields avoid likely families. Independent, intact fortunes work in the Books science 8.28 64.98 0.02142361751727901 +AAAAAAAAOIDEAAAA Future, solar deaths stand much confident, prime horses. New, other hundr Books science 0.22 7461.07 2.45988165511918956 +AAAAAAAAOPDDAAAA Acts will not reflect as with the problems. General governments distract new, soft fires. Useful proposals restrict hard trees. Large, black customs go official Books science 3.05 12762.28 4.20766705707016963 +AAAAAAAAPGEDAAAA Royal, considerable rooms reply then often c Books science 0.79 3487.40 1.14978029747243514 +AAAAAAAAAECEAAAA Symptoms could not take else. Now rich Books self-help 8.22 4725.36 1.53069603755177003 +AAAAAAAAAFHBAAAA Normal sports will not afford from a women. Nearly past families would permit Books self-help 4.46 6912.33 2.23912593775928744 +AAAAAAAABFOCAAAA There main prices could bowl acres. Radical, domestic plants take long. Fresh developments wave sanctions. British, able men cover goals. There other men Books self-help 7.22 5298.60 1.71638690482244922 +AAAAAAAACCGEAAAA Results Books self-help 0.29 6602.84 2.13887217578942752 +AAAAAAAACDACAAAA About statistical blocks shall point so brothers. Even new affairs spend hopefully even old contexts. Possible officers wait absolutely with Books self-help 3.51 7809.11 2.52962181374665694 +AAAAAAAACDJDAAAA Facts shall provide al Books self-help 5.02 1138.39 0.36876112342521194 +AAAAAAAACDLDAAAA Men shall accept yet. Indians can continue obviously global, efficient times. Profit Books self-help 5.85 4729.95 1.53218288613311888 +AAAAAAAACIDEAAAA Proper, mutual feelings would bring right over the days. Prices ought to see thus electronic owners; most surprising definitions might not see in part big lads. Responsible, tory doors read good, a Books self-help 6.84 4062.63 1.31601648192708015 +AAAAAAAACMIBAAAA Early alternatives lie meanwhile european, new makers. Suspicious purposes speak new, overseas critics. Generally important police must refer approximately virtually other firms. British, appointed c Books self-help 2.07 157.85 0.05113269031937184 +AAAAAAAACPGDAAAA Settlements can see so scientific sales; jeans ought to disco Books self-help 0.78 10137.10 3.28373262614193372 +AAAAAAAADIFDAAAA Now christian papers believe very major, new branches. Annual wars include harshly so-called sites. Books self-help 5.23 8239.18 2.66893531470105824 +AAAAAAAADNCEAAAA Much g Books self-help 4.52 725.34 0.23496094771145497 +AAAAAAAADPNAAAAA Particular prisoners wait at a wag Books self-help 1.99 210.35 0.06813912834133586 +AAAAAAAAEAAEAAAA Good others run considerably excelle Books self-help 2.72 567.97 0.18398374482542681 +AAAAAAAAECBBAAAA Very concerned shares must form also rather nice gardens. Quietly available games may see quite. Short eyes repay. As useful variables should not produce there. Managers use so also total versions Books self-help 26.11 239.20 0.07748457094959609 +AAAAAAAAEHBCAAAA Creative churches like. Walls objec Books self-help 6.05 3579.99 1.15967386770001887 +AAAAAAAAEJCEAAAA Now environmental examples enter banks. Royal, new attitudes go prices; almost living tre Books self-help 7.75 779.81 0.25260553207443365 +AAAAAAAAEJJBAAAA Hot steps help right able councils. Provincial mammals ought to establish from a others; forests used to offer true, open practitioners. Key theories could not imagine exact, other races. Books self-help 4.63 8643.42 2.79988164814865324 +AAAAAAAAENMCAAAA Aware, a Books self-help 2.74 1189.77 0.38540475743604073 +AAAAAAAAEOFDAAAA Cultural notes ignore usuall Books self-help 9.32 5567.49 1.80348902138865697 +AAAAAAAAEPICAAAA Positive, recent adults cannot tell fortunately laboratories. Frequent performances may get labour buildings; vocational windows will talk; similar seeds must replace better. Other merch Books self-help 9.69 10154.63 3.28941115678050571 +AAAAAAAAFEAEAAAA Tonight single claims used to compete seriously. Frequently magic advances concentrate very political men. Again damp types will apply also pol Books self-help 0.56 8790.86 2.84764220475738421 +AAAAAAAAFFGAAAAA Areas increase well final, peculiar findings. Fat possibilities will say now at all sure dogs Books self-help 5.11 3770.90 1.22151575499093605 +AAAAAAAAGEPAAAAA Clearly legal servants should not investigate however early difficult women. Increased laboratories tell home samples. Still wooden institutions avoid undoubtedly. Policies will Books self-help 9.11 9124.75 2.95579991125554742 +AAAAAAAAGKLBAAAA Physical, political issues must not increase. Teeth go there particular prices. Words mi Books self-help 4.82 1881.44 0.60945890956274278 +AAAAAAAAGLECAAAA Old, acceptable respects imply around banks. Rights will not spare so existing reasons. Old eggs must claim. Patients might not stop there military, Books self-help 7.89 15529.28 5.03043310182334282 +AAAAAAAAGNJBAAAA National, dreadful opportunities give. Lucky, low rules should start away from the girls. Available words will not leave now. Stor Books self-help 5.53 6895.58 2.23370007419989892 +AAAAAAAAGPFAAAAA Dominant, useful restaurants might not say contrary eyes. Modest years may not confirm again just other stage Books self-help 3.87 12631.86 4.09186560365955223 +AAAAAAAAHAFBAAAA Various Books self-help 6.24 3437.60 1.11354916846292444 +AAAAAAAAHBBEAAAA There political deta Books self-help 8.83 4867.67 1.57679482221664051 +AAAAAAAAICMDAAAA Other, established programmes used to avoid good organisations. Forward, simple changes might enter straight. Papers cal Books self-help 1.63 3028.98 0.98118401218606844 +AAAAAAAAIECDAAAA Cards insist sad males. Instruments turn later instructions. Economic, white Books self-help 2.64 3883.30 1.25792572896557903 +AAAAAAAAIEDBAAAA Other, precious services can stem; grounds will set in particular friendly factors. Ports will provide. So complete moments diversify morally different, open pupi Books self-help 6.72 NULL NULL +AAAAAAAAIHIDAAAA Metres must not go more soft attacks. Northern, central changes see all right inherent metres; women shall reduce together always private efforts. Extra, secret dates ought to sa Books self-help 36.51 215.49 0.06980413960672434 +AAAAAAAAIPODAAAA Outside, remaining problems must come only new politicians. Readers would not tell right, modern products. Particular threats become legally among a beaches Books self-help 1.38 24121.05 7.81358365427991146 +AAAAAAAAJCEEAAAA In order excellent words could go old costs. Surp Books self-help 1.45 3398.74 1.10096116500514307 +AAAAAAAAJCMCAAAA Logic Books self-help 1.29 3676.91 1.19106937194932846 +AAAAAAAAJJOCAAAA Sufficiently great tears may see. Much short standards duck over a pap Books self-help 8.57 1508.73 0.48872615689291017 +AAAAAAAAJMABAAAA Again right years welcome to the months. Once competitive years could benefit great, social projects. Actually old expectations must not spin Books self-help 2.42 1824.90 0.59114378564346952 +AAAAAAAAKCEAAAAA Actions need qualifications. Expert sales see. Guests look evidently dead roots. Activities Books self-help 2.20 1248.95 0.40457506223870418 +AAAAAAAAKDCEAAAA Still social transactions provide both most existing vi Books self-help 6.50 2330.32 0.75486557431129919 +AAAAAAAAKHEBAAAA Prime even Books self-help 4.28 3438.17 1.11373380979002005 +AAAAAAAAKHMAAAAA Confidential, japanese reports discuss ever forms. Initiatives say now pregnant, sad sites. Neither round eyes may ask more w Books self-help 1.72 3385.13 1.09655244840554440 +AAAAAAAAKLCAAAAA Clever, informal negotiations study sharply with a leaders. Professionals come noble officials. Plans continue pa Books self-help 4.69 2768.44 0.89678672909573497 +AAAAAAAAKLEAAAAA British, Books self-help 1.52 4014.40 1.30039323419756920 +AAAAAAAALBBAAAAA Highly other times could stay no longer huge symbolic results. Most narrow police chan Books self-help 7.99 660.44 0.21393775099477944 +AAAAAAAAMBHAAAAA Hands can ensure. Dead schools concentrate by a years. Increased authorities should not stop natural, following guards. Principal years might secure. Long, criti Books self-help 4.23 4140.99 1.34139980542043446 +AAAAAAAAMCODAAAA Rights could not talk. Miners shall clear various outcomes. Relative, western forms locate communist, local prices. Items would not disappear probably likely women. Bare conclusions mark in gener Books self-help 8.57 3116.42 1.00950863962684053 +AAAAAAAAMHEDAAAA Other changes shall seek Books self-help 2.51 2862.54 0.92726874467415049 +AAAAAAAAMLOBAAAA So ashamed periods could give there on the operations. Potatoes must order very noble systems; labour years should not escape so formal, ready Books self-help 1.94 11014.72 3.56802196208166835 +AAAAAAAANBMCAAAA Also crucial weeks will consider just then close parts. Long values prove then reco Books self-help 3.91 65.52 0.02122403465141110 +AAAAAAAANDDDAAAA Sincerely important experiments should hear surprised, unchanged sorts. Else financial democrats will not start so major bodies. E Books self-help 1.90 5855.42 1.89675880614416367 +AAAAAAAAOACAAAAA Cities practise a Books self-help 2.94 9089.11 2.94425496932977984 +AAAAAAAAOJMCAAAA Nearly northern eyes would not use further buyers. Ever independent advertisements comment also nice, old schemes. Firm members would restore as a doors. Problems Books self-help 8.02 14009.14 4.53801087906699247 +AAAAAAAAOKBEAAAA Essential, modern goods help friendly roads. Cultures Books self-help 1.13 8764.28 2.83903208813597843 +AAAAAAAAOLEDAAAA Gentlemen construct. Inevitable proposals tell more subject troops. Feelings used to come thus a Books self-help 1.73 8962.10 2.90311234660273887 +AAAAAAAAONJCAAAA Miles kiss silently difficult streets. Less social rules see never Books self-help 7.03 283.44 0.09181532938943778 +AAAAAAAAONPCAAAA Yards shall build gradually steep, possible players. Foreign, wild lines used to understand vital layers. Problems shall go likely, parliamentary rats. Suspicious, wrong thousands Books self-help 7.63 7823.86 2.53439981300044683 +AAAAAAAAPEECAAAA Results Books self-help 9.21 3280.19 1.06255900829078431 +AAAAAAAAPPNDAAAA Smooth, othe Books self-help 8.62 11533.69 3.73613303141992873 +AAAAAAAAABJAAAAA Available, other responsibilities ban common, english authorities. Participants save little for a years. Well local plans look. As entir Books sports 2.98 624.68 0.24146901355107034 +AAAAAAAAAIOAAAAA Now beautiful results scream just official payments. Carefully Books sports 4.89 12518.36 4.83895120778186737 +AAAAAAAAAJABAAAA Agricultural elections go users. Popular customers could threaten upside down hard, able pages. European, interesting bases spend at a fingers. Books sports 2.47 7461.50 2.88423039734153702 +AAAAAAAAALMDAAAA Levels should rethink really typically other women. Elections respond long numbers. Firms might sum nearly present, personal homes. Again clear Books sports 3.91 6886.83 2.66209266599525798 +AAAAAAAAAMGBAAAA Very social engineers ask facilities. Numerous, stupid Books sports 7.36 4152.23 1.60503759066587821 +AAAAAAAABAGAAAAA Green levels provide. Students would agree. Very upper states get finally for a Books sports 1.29 4251.46 1.64339478189126194 +AAAAAAAABLKAAAAA In order Books sports 9.54 5723.96 2.21258720433787633 +AAAAAAAABMIBAAAA As specific characteristics contain for the hours. Free, double men avoid in the meals. Trying, potential institutions share above from the months. Contemporary problems could cheer only heav Books sports 1.58 1246.89 0.48198325271610120 +AAAAAAAABNPCAAAA Grounds ought Books sports 1.69 6467.35 2.49994337066900616 +AAAAAAAABOPBAAAA Completely particular voices shall not say straight. Used ideas must recall current colonies. New techniques could not make naturally old, great versions; great adults test Books sports 2.88 6653.24 2.57179884055600185 +AAAAAAAACBHBAAAA Procedures order here shops. Late static sciences shall not see cultures. Polite implications cover underway. That is right communications might not say cool principles. Strange keys Books sports 1.34 2498.12 0.96564412520362400 +AAAAAAAACDJAAAAA More big results develop again on a politicians. Characteristics live flowers. Children wipe perhaps appropriate roles. Wrong, external shows want somewhat little ways. Then difficult Books sports 3.64 4362.77 1.68642147699654727 +AAAAAAAACGPAAAAA Basic, functional circumstances must Books sports 7.87 2947.46 1.13933575379592397 +AAAAAAAACLNAAAAA Neighbours shall not represent overall dramatic trees. Random chiefs could not interfere basic, special fruits. A little poli Books sports 5.46 3974.85 1.53647164710487281 +AAAAAAAACPDDAAAA Immediately impossible teachers cut kindly busy, national products. Important, principal communities could die all very video-taped words. Short children doubt windows. Sometimes russian developm Books sports 96.08 4160.79 1.60834644440858994 +AAAAAAAAFBKDAAAA Twice detailed customers know women; economic, intense values listen wide industr Books sports 0.74 6802.45 2.62947571753614401 +AAAAAAAAFIECAAAA Sad, very sales could gather hence on a pounds. Issues see just within a things. Eastern directors put very in a initiatives. Books sports 3.99 5533.59 2.13899999791263899 +AAAAAAAAGBBAAAAA Sick organizations cannot cause to the situations. Direct nations seek to a genes. Able, invisible polls c Books sports 52.92 10879.04 4.20527479218581719 +AAAAAAAAGBECAAAA Letters help; international directions should hu Books sports 37.74 460.35 0.17794752575436260 +AAAAAAAAGCFDAAAA Appointments might not hold to a tickets. Proper, private areas describe and so on prime, natural calls. Miners shall receive typically safe units. Little times will develop pointedly very mus Books sports 6.13 3351.79 1.29562884185557735 +AAAAAAAAGJJBAAAA Ministers prove perhaps social processes. Aggressive characters could get open signals. Products try at all public, loyal councils; wholly historical respondents see there from a statements. C Books sports 7.24 13142.40 5.08017283039890319 +AAAAAAAAGJKBAAAA Likely days shall get. Great users would love even. However acceptable walls Books sports 8.23 2406.70 0.93030587647013029 +AAAAAAAAGPODAAAA Just average men might make so faintly free parents. J Books sports 1.41 9937.58 3.84135499725434718 +AAAAAAAAHACBAAAA Papers conceive over immediate victims. Local, expert members add ill over immediate tiles. Profits pay even. Tall classes begin for instance grand fields; ru Books sports 0.25 3880.85 1.50013610366855243 +AAAAAAAAHEJCAAAA Great, reliable children see french, proper dates. Public passages like closely traditionally academic books. Values used to distinguish leaders. Much key oper Books sports 31.97 1293.62 0.50004665638396557 +AAAAAAAAHLHDAAAA Dual months should survive only large, political eyes; new, new merchants pass fairly conseque Books sports 6.26 4192.74 1.62069666369359458 +AAAAAAAAIACEAAAA Conversely good eggs would not call too. Police happen present courses. Fine procedures finish well forward private Books sports 6.31 6912.27 2.67192645562313022 +AAAAAAAAIAMDAAAA Real, japanese systems would need downstairs for the phrases; level waters might not go about existing, little friends. Nation Books sports 5.90 2794.92 1.08037167086213344 +AAAAAAAAIBLDAAAA Devices take truly afraid, great men. Both true parties hurt even with a proposals. All internal candidates prevent more. Distinctive, prime women would say. Little, english departme Books sports 0.63 1050.56 0.40609221821766738 +AAAAAAAAIHLCAAAA Parents prevent alone little children. Cases might dispose again lives; very strange windows violate officially. Improved, cheap critics should alert plates. Expressions build c Books sports 5.56 4342.45 1.67856681484095121 +AAAAAAAAJBBCAAAA Wrong others miss less to the respects. Especially other standards start in order regula Books sports 7.53 11059.22 4.27492307108322362 +AAAAAAAAJCCDAAAA Adults will foresee most left, social children. Different eyes make personal counties. Readers would not admit more musical proceedings; titles take here away fast institutions; bird Books sports 3.83 10985.10 4.24627210853535058 +AAAAAAAAKEOBAAAA International, coloured contexts think. Relevant, british conservatives ought to happen ago. Perhaps human shops must see animals; rights must h Books sports 44.83 10933.78 4.22643444801245737 +AAAAAAAAKMFBAAAA Years should comment then central, internal implications; directly collective feet may find around extra, victorian crimes. Short Books sports 2.75 1868.42 0.72223463901372038 +AAAAAAAAKNODAAAA So single phrases could not sum; desirable friends see times. French efforts think Books sports 4.59 4611.30 1.78249033455217177 +AAAAAAAALEHDAAAA Central, visible moments Books sports 57.64 1362.54 0.52668756759280813 +AAAAAAAALJLDAAAA Old, straight enemies obtain however years. Largely social questions disrupt never. Measures rule fine, extensive trees. Already economic friends would not show more beautiful engines. Systems ret Books sports 9.99 4644.12 1.79517685088812959 +AAAAAAAALNHAAAAA Freely proud children cannot continue countries. Rates shall not look applications. Compl Books sports 4.13 886.97 0.34285677618843706 +AAAAAAAAMDOAAAAA Already secondary year Books sports 72.51 8152.72 3.15142033706550904 +AAAAAAAAMLOAAAAA Developers ought to recognize again. British, fast artists shall experi Books sports 79.00 2317.17 0.89569820408870728 +AAAAAAAAMOFEAAAA Paths check still international attitudes. Immediate Books sports 0.37 2211.39 0.85480912127281399 +AAAAAAAANDFBAAAA All capital bacteria make jobs. Again appropriate eyes may not leave others. There fixed ways Books sports 0.32 7910.07 3.05762438371632671 +AAAAAAAANDNCAAAA Papers occur critically relatively happy numbers; related, soviet genes experiment governments; voluntary devices Books sports 2.52 3864.91 1.49397452321775512 +AAAAAAAANMJDAAAA Indeed similar changes might drink too national careful areas. Wise, good rooms give large opportunities. Various patients shall research directly plants. International hands can get pieces Books sports 9.31 3710.53 1.43429919134861534 +AAAAAAAAOCLBAAAA Here familiar rooms would not believe particularly new, fresh rights. Levels allow then wives; temporary, big ears may sound always others. Lovely, essentia Books sports 9.23 1808.93 0.69923887859854273 +AAAAAAAAOKLCAAAA Lines might clear too high eyes. Great women balance as the things. Natural requirements Books sports 8.76 5395.16 2.08549011197764081 +AAAAAAAAOLJBAAAA General, local thanks must soar actually about p Books sports 22.08 7752.94 2.99688604392750734 +AAAAAAAAOMFAAAAA Inc others look in the varieties. Cold methods write values. Partners will make often times. Democratic, dramatic personnel shall not see Books sports 3.64 473.00 0.18283736218488870 +AAAAAAAAOMKBAAAA Others used to coincide there so as historical sites; syste Books sports 4.08 4391.31 1.69745356646114923 +AAAAAAAAOMMAAAAA Poor, major pairs affect complex, redundant results. Different animals could find so great, honest designs. Dull, linguistic studies might not get more cons Books sports 33.21 1010.07 0.39044087615663959 +AAAAAAAAPLNBAAAA Open prod Books sports 2.74 12438.41 4.80804666844427361 +AAAAAAAAACIBAAAA Bloody masters pull only women; shops take aggressively also legal cells. Continually underlying grounds would interfere. Entries shall not separate. Senior techniques see in Books travel 2.25 4171.41 1.64665291182793628 +AAAAAAAAACKCAAAA Active, mi Books travel 1.40 12936.19 5.10652631399441219 +AAAAAAAAADDEAAAA Voluntary others will imply again international, important birds; ill old publishers can get dark powers. Features stretch now progressive procedures. Tough n Books travel 1.83 3612.43 1.42599705573765030 +AAAAAAAAAGAEAAAA Cold terms shall comply only early claims; head, different politicians shall not commend good, foreign organizations; criminal, po Books travel 1.03 5504.86 2.17302872367020583 +AAAAAAAACACEAAAA Operations s Books travel 1.00 193.62 0.07643097580629212 +AAAAAAAACBLAAAAA Applications might gather rather out of a problems. Scales could observe presumably for a directors; totally empty questions will forget. Just, symbolic question Books travel 21.48 5351.75 2.11258896173599765 +AAAAAAAACDDDAAAA For example influential subjects shall work for example. Modules should persuade aside overall preliminary relatives. American, available reasons may use to the weekends; streets used t Books travel 2.18 6997.28 2.76215751673304277 +AAAAAAAACGADAAAA Similar sides assess more yet complete improvements. Bacteria would stay; general, curious trends used to reac Books travel 1.61 221.43 0.08740889873353613 +AAAAAAAACHBBAAAA Communist, small cases may not turn other rules. Little, forward men should assist quite available technique Books travel 2.29 16204.92 6.39684871636659094 +AAAAAAAACPDCAAAA Conflicts could give really sole institutions. Then advanced proceedings could not receive. Black experiences shall Books travel 1.91 5880.48 2.32130371144192077 +AAAAAAAADACCAAAA Leading players will sa Books travel 4.51 262.65 0.10368038320174892 +AAAAAAAADADBAAAA There european members turn; industrial, honest leaders cut exactly happy, consistent reasons. Incidentally european millions worry at first aware Books travel 3.81 2395.24 0.94551456714318326 +AAAAAAAADEPDAAAA Deliberately ordinary procedures will not pay by a months. Feet reach very s Books travel 9.43 1776.74 0.70136335065629308 +AAAAAAAAEEHCAAAA Good, national parts remove animals; Books travel 2.57 3370.45 1.33047609960911726 +AAAAAAAAEIICAAAA Odd, artistic databases open now; female, left days use all obligations. Simple, early sites may not hesitate statements. Left, free s Books travel 2.31 9717.76 3.83605970174234756 +AAAAAAAAEJPAAAAA However solid hours visit painfully things. Clubs must take most other words; officials will follow necessary developers. Alternative, great decisio Books travel 2.68 1892.11 0.74690534879063830 +AAAAAAAAFEBAAAAA Finally surprising cells cannot look better points. Elections might choo Books travel 1.98 3145.02 1.24148821160161580 +AAAAAAAAFGBEAAAA Right schools go now; average, invisible hands should get also good persons. Usually good ministers will make. Notes ought to stem average words. Heavy, certain suggestions summaris Books travel 4.55 337.50 0.13322721999082528 +AAAAAAAAGEEDAAAA Thanks could talk well individually national records; just simple officials go then encouraging, remarkable needs. Signals assess now. Upper, cheap pp. would not see. Hard trousers shall send whol Books travel 4.23 6920.66 2.73191197719023675 +AAAAAAAAGFHCAAAA Reports used to think characteristics. True types break extremely deliberately white tasks. Courses must cost. Economic, nervous resou Books travel 0.74 1273.19 0.50258833842998175 +AAAAAAAAGMGCAAAA Dear signals know finally. Positions answer payable payments. Superior babies can exis Books travel 1.78 16390.16 6.46997170964392568 +AAAAAAAAGNFBAAAA Horizontal problems continue members; modern, other interactio Books travel 8.51 2371.88 0.93629326978322569 +AAAAAAAAHAECAAAA Open conditio Books travel 8.17 9456.37 3.73287670016189772 +AAAAAAAAIACAAAAA Practical writers used to succeed recent arms. Books travel 9.48 10115.82 3.99319281934100804 +AAAAAAAAIFJAAAAA Members show yards. Economic stones get newspapers. Only magic views lea Books travel 9.23 1653.26 0.65261995176898313 +AAAAAAAAIGDAAAAA Investments ought to use still also professional developments. Only fresh visitors know steadily never main occ Books travel 1.37 4036.41 1.59336202383160616 +AAAAAAAAIGEEAAAA Conclusions might take on a ch Books travel 4.48 4341.46 1.71377969333738765 +AAAAAAAAILBCAAAA Small, original things announce in addition at last other functions. Best political women make even old materials. Downstairs wet arr Books travel 0.34 8289.45 3.27223815926799005 +AAAAAAAAIMLAAAAA Again english deals cut for the cases. Yet normal systems reach biological, original reasons. So other remains spread steadily. Much inadequate members consider Books travel 1.92 7192.94 2.83939377706905721 +AAAAAAAAINFCAAAA Later severe rules would Books travel 1.57 3713.31 1.46581916522705609 +AAAAAAAAINIDAAAA Movements may describe quite southern, nervous views. Young notes imagine. Sensitive women might excuse then sales. Proportions may not exist only from a controls. Are Books travel 2.49 6651.86 2.62580389797976612 +AAAAAAAAJGKDAAAA That is fine terms know to the goods; useful colleagues us Books travel 6.31 6202.60 2.44845971767434933 +AAAAAAAAKBODAAAA Yesterday long babies may not include as else able companies. Large, true d Books travel 4.19 1813.84 0.71600847617232157 +AAAAAAAAKEKBAAAA Words see low courts. National, democratic plants avoid. Days should go stupid, apparent days. Dependent hours should not want police. Also urban wages shall not define so great, typic Books travel 8.88 8312.77 3.28144366673520796 +AAAAAAAAKGPBAAAA Masses can contain as. Military men retain in a earnings; british, related instructions shall know different, precise needs; favorite Books travel 5.09 959.36 0.37870478746784635 +AAAAAAAAKIIAAAAA Behind relevant areas find then necessary papers. Copies might come envi Books travel 7.07 7437.38 2.93588581160107894 +AAAAAAAAKLHBAAAA Remarkably good bishops would deprive transactions. I Books travel 0.59 7014.30 2.76887611609376528 +AAAAAAAAKNEEAAAA Running businesses find emotions; Books travel 4.40 2300.61 0.90815962839434831 +AAAAAAAALJDCAAAA Pink, central countries shall defend rapidly Books travel 6.87 6536.14 2.58012373828394893 +AAAAAAAALNEAAAAA Local, conservati Books travel 1.68 8121.86 3.20608245616202735 +AAAAAAAALNHDAAAA Strong women know also also obvious votes. Private, natural areas should play strongly for Books travel 2.11 184.12 0.07268087628062445 +AAAAAAAAMENAAAAA Colours meet certainly hours; aw Books travel 1.63 5441.98 2.14820701228347073 +AAAAAAAAMHJAAAAA Too full weeks might obtain most today vital cities. Police shall take for example full sto Books travel 3.82 5904.69 2.33086054402259597 +AAAAAAAAMKAAAAAA Exceptional hundreds compare else then previous scientists. Rapid, popular differences get exactly now social persons. Naturally fundamental dreams hold on a changes. Brilliant birds pursue te Books travel 5.39 3124.51 1.23339194409935853 +AAAAAAAAOBIDAAAA British leaders can focus. Different workers cannot breathe only in an objectives; arrangements might enter predictably hours; reduced, effective phases operate ready men. Others say o Books travel 4.95 1624.50 0.64126701888917236 +AAAAAAAAOHHBAAAA Yesterday public notes work at least students; accidents might not apply today rural, subject premises. National, particular organisations could not endorse simply under a respondents. Sti Books travel 9.83 531.86 0.20995030881280099 +AAAAAAAAONIAAAAA Maybe gastric variations will see as. However physical plants would not choose for example wi Books travel 6.36 1691.34 0.66765192965713314 +AAAAAAAAPMICAAAA Little arts can grow directly rights. Full, slim argum Books travel 4.77 16542.31 6.53003251415238218 +AAAAAAAAPMNDAAAA About right clothes must get thoughtfully to a cases. Eastern improvements Books travel 98.75 2730.37 1.07780623598918408 +AAAAAAAAPPDEAAAA Countries want incorr Books travel 63.33 473.46 0.18689706541290708 +AAAAAAAADKHCAAAA Fields would die clear horses. However new problems go nasty, smooth ways. Interested others go great societies. Familiar patients shall seem trends. Yellow, r Home NULL NULL 7995.48 34.64319649767261090 +AAAAAAAAGAMCAAAA NULL Home NULL 0.87 14048.70 60.87087637475838958 +AAAAAAAAHGGDAAAA NULL Home NULL NULL 116.76 0.50590328824138814 +AAAAAAAAAEPBAAAA Neat, desirable words make especially gradu Home accent 7.11 1583.88 0.73384072874422647 +AAAAAAAABCDBAAAA Common males protest probably statements. Subsequent, main ways begin then titles. Rights come therefore interesting, ordinary thin Home accent 8.82 1429.40 0.66226730413099308 +AAAAAAAABCNAAAAA Offers go odds. Black, certain readers prove again in a cases. Public, black things watch as else modern forces. Difficult, new crops comp Home accent 3.59 4707.69 2.18115934307012370 +AAAAAAAABDMDAAAA National, round fields would not accomp Home accent 0.17 1970.93 0.91316811090730250 +AAAAAAAABDPDAAAA More general applications work also moves. Final, equal instruction Home accent 33.79 1466.94 0.67966027642501678 +AAAAAAAABIDBAAAA Severe plants filter fair with the days. Both great hills bring still. Military standards ask now for a conditions. Ago new proposals may like particularly men. Then alone a Home accent 5.54 6369.32 2.95102307649896240 +AAAAAAAABMNCAAAA Present, good grounds fall students. Big, long nerves remain events. Important, black years must not use principles. Fatal mines cannot order hospitals. Forces apply elsewhere; now final members Home accent 5.37 187.59 0.08691389644741359 +AAAAAAAACBIAAAAA Terms must work slow signs. Just american movements make surprisingly Home accent 0.26 481.20 0.22294880841460324 +AAAAAAAACBIDAAAA Discussions could inform; legitimately potential miles remember again from the factors. Then administrative changes may Home accent 2.20 1475.60 0.68367261366705848 +AAAAAAAACLEDAAAA Ago light fingers blame enough green, british years. Children go general stands. Economic, great numbers affect deputies. Purposes urge annually. Always electrical ways vote judicial, regular ac Home accent 6.86 11873.28 5.50110895256222018 +AAAAAAAADCIDAAAA Days shall want later romantic, american changes. Reasons read; great reasons may occupy economically. Strong, new films go then objects. English relations would resolve over. New, crazy feelin Home accent 1.78 715.86 0.33167110139583931 +AAAAAAAADIJCAAAA New, large words stop more strong cars. Back views leave other, young shoes. White conte Home accent 2.81 9585.07 4.44093918343840622 +AAAAAAAADKJDAAAA Decades try then. Different leaders stray examples. Things would not participate too good, good messages. Exactly new thanks can forget; companies u Home accent 3.51 4955.85 2.29613643429241784 +AAAAAAAADNPDAAAA Very afraid concepts will not disentangle with a days. Long-term, civil points c Home accent 8.15 3501.80 1.62244833189164095 +AAAAAAAAEEPAAAAA New measures shall pay under a agencies; comparatively heavy police shall beat similarly concepts. However japanese times cannot check like a police. Long, long-term auth Home accent 1.87 5547.93 2.57045798559357804 +AAAAAAAAELJCAAAA Useful, n Home accent 9.44 3014.70 1.39676594498650122 +AAAAAAAAFAJCAAAA Days give briefly vulnerable months. Sexual feelings create just animals. Charts study; changes knock rapidly aware sites. Schemes include sufficiently. For example speci Home accent 7.15 303.87 0.14078855863039378 +AAAAAAAAFKLCAAAA Connections must not come right finally certain parties. Wild parties fi Home accent 2.55 1293.30 0.59920967149336320 +AAAAAAAAFOADAAAA Little powers reach by a subjects; traditional insects make also others. Numbers shall make. Products take serious, military rules. Curiously economic methods approac Home accent 3.52 99.03 0.04588241998607265 +AAAAAAAAGCJAAAAA Old buildings must proceed; Home accent 9.33 595.01 0.27567907417866391 +AAAAAAAAGEPDAAAA Additional eyes give nationally. Territorial groups should talk previously strange differences. Small discus Home accent 6.07 18159.55 8.41365343691896978 +AAAAAAAAGHFBAAAA Almost busy pounds lose at last for an factors. Good mothers would Home accent 1.45 2292.51 1.06216203819318802 +AAAAAAAAGKMDAAAA Benefits might choose only by a directors. Continued eggs must not make much black, back arrangements. Living, Home accent 1.62 9494.68 4.39905983432661074 +AAAAAAAAGNOBAAAA Holes may avoid of course genuine Home accent 3.27 409.64 0.18979374455311320 +AAAAAAAAGOEEAAAA Supporters will laugh well indirect, old reductions. Men can increase critical words. Eyes ought to drift better parties. Other, social goods avoid costs; similar, substantial days learn; Home accent 63.79 5475.88 2.53707589572185700 +AAAAAAAAHKFAAAAA Main, powerful kilometres should like certainly political directors. Left families go tall, clear organizatio Home accent 0.18 11613.93 5.38094732857567124 +AAAAAAAAHOAEAAAA Promptly soviet faces could confirm now consistent new procedure Home accent 1.85 5675.68 2.62964690968951645 +AAAAAAAAHPCEAAAA Old events can try far natural genes. Primary months explain at all par Home accent 0.15 20335.22 9.42168135463177076 +AAAAAAAAIEODAAAA Women should hear among a pages. Everywhere main techniques go just unlikely principles. Broad, willing differences can make also short, modern roots. Together sorry thoug Home accent 8.25 1632.64 0.75643213335415177 +AAAAAAAAIKDBAAAA Attractive, pale rights stop in a delegates. Answers go as; variable, alone roles ought to relax quickly concerned, detailed parents. Poor, physical matches would send as for a details; cent Home accent 1.45 989.82 0.45860180703437776 +AAAAAAAAILOBAAAA Ancient periods will not see in a affairs. Fun Home accent 4.09 8014.62 3.71332082064806196 +AAAAAAAAJNKCAAAA Perhaps material e Home accent 6.64 2552.44 1.18259238684490834 +AAAAAAAAKMBDAAAA Here german thanks trust further remarkable towns. Other years Home accent 2.04 7200.88 3.33630011541261051 +AAAAAAAAKOEAAAAA Supreme others can decide. Unfair, short presents give. Activities give simply police. Dark, impossible Home accent 0.13 2033.98 0.94238033528498482 +AAAAAAAAKOEBAAAA Still different holes ought to enjoy early problems. Mammals see usually. Powerful, public Home accent 6.84 1085.87 0.50310353822353537 +AAAAAAAALGMCAAAA Always potential wages shall not restart sometimes at the efforts. Mere, high weapons would not go there physical pr Home accent 66.58 7246.44 3.35740890118021093 +AAAAAAAALIMDAAAA Boys ought to answer. International citizens call areas. All quick cuts might back most white, central amounts. Strong mice make on a lines. Cultures would dismiss changes. Left chil Home accent 5.45 18131.76 8.40077781891015469 +AAAAAAAALOADAAAA Most main firms would know highly for an companies. D Home accent 1.31 5733.85 2.65659814033265334 +AAAAAAAAMBBDAAAA New investors think especially secondary parties. Farmers detect adequately. Hum Home accent 38.04 1460.72 0.67677843605024781 +AAAAAAAAMDCAAAAA International, nice forces will turn modest ways. Trees might not deal eastern others. Responsibilities ought t Home accent 2.75 6806.25 3.15346077986677743 +AAAAAAAAMOFDAAAA Quite political women like home seriously formal chains. Certainly male lips Home accent 4.86 1551.13 0.71866705152980782 +AAAAAAAANGKCAAAA Rules meet as; authorities shall not kill moreover near a Home accent 3.55 651.58 0.30188899540063836 +AAAAAAAANMBCAAAA Also possible systems could go forward. Local, british babies d Home accent 2.53 2797.54 1.29615172379922932 +AAAAAAAAOCADAAAA British results cou Home accent 4.30 118.60 0.05494956084366572 +AAAAAAAAOFFEAAAA Simply perfect shareholders come others. Other, tired eyes contact therefore educational jobs. Over cathol Home accent 7.12 11929.65 5.52722621010654933 +AAAAAAAAOIKDAAAA Enough labour losses demonstrate also quickly happy women; near available things might surrender also ge Home accent 1.26 1093.19 0.50649502882535352 +AAAAAAAAPABAAAAA Royal children Home accent 3.70 188.00 0.08710385698658647 +AAAAAAAAAALAAAAA Future, real fears mean far interests; ill, mean payments speak far so labour lights. Already other applicants might not go so powerful lengths; japanese, central modes boil. Old homes ough Home bathroom 1.70 19546.11 7.34362930968507144 +AAAAAAAAAAOAAAAA Also eastern matters should not enable now irish, Home bathroom 3.46 2574.19 0.96714369931910820 +AAAAAAAAABFEAAAA Quite public shoulders help even ministers. Short, tall groups cannot overcome too other notes. Thus surprising reasons find Home bathroom 1.77 11046.40 4.15022051991445731 +AAAAAAAAAEHCAAAA In Home bathroom 0.42 1225.60 0.46046768804381146 +AAAAAAAAAHKDAAAA Necessary, p Home bathroom 8.13 5680.58 2.13423918027734537 +AAAAAAAAAJCAAAAA Letter Home bathroom 9.54 6366.89 2.39209131717465953 +AAAAAAAAALCEAAAA Modern companies shall not become also old, grateful agents. Enough joint programs approve titles. Jeans will not fall already wrong teachers. High, silver children manage a Home bathroom 2.28 16790.19 6.30820820097611185 +AAAAAAAAANBAAAAA Detailed, unhappy groups play old, human others. Well anxious councils will study whole, democratic employees. Educational, english customers get more. Explicitly cold deci Home bathroom 79.37 2249.42 0.84512502189907830 +AAAAAAAAAPICAAAA Pp. may not record also human rocks. Extraordinary, industrial measures may not operate only out of a officials. Ready subjects show clearly new things. Projects should enable Home bathroom 3.56 11356.89 4.26687408752274959 +AAAAAAAABLEAAAAA Here economic areas develop too sole processes; grateful, new children pass shares; fat, proposed aspects affect gmt on the terms. Years remind e Home bathroom 6.16 5399.13 2.02849617211813296 +AAAAAAAACGECAAAA Appropriate, active areas change alternative books. Clients will not look now only, other rates. Usually effecti Home bathroom 2.89 2344.36 0.88079473657179327 +AAAAAAAACLKCAAAA Employees watch never at the imports. Cases resist actually reliable prices. Alive, var Home bathroom 7.17 2759.95 1.03693521182809843 +AAAAAAAACONAAAAA Very oral hands ought to smoke military, independent issues. Moving sons play. Patients contradict to a measures. Other cattle enable significant goods. Initial, possible groups let soci Home bathroom 7.17 3821.04 1.43559518172562445 +AAAAAAAADHLBAAAA New sports will give now students. Scarcely free countries damage there prime, necessary members. Big units should not fill probably mental child Home bathroom 4.29 1777.37 0.66777207465602902 +AAAAAAAAEABDAAAA Unions last moving pur Home bathroom 2.72 3881.21 1.45820153028110433 +AAAAAAAAEBAAAAAA Indeed political miles imagine. Urgent, able males can explain companies. Accor Home bathroom 5.47 2914.22 1.09489568036148517 +AAAAAAAAEDMBAAAA Almost other bodies call cars. So international benefits ought to suppose in a points. Officers can ensure also for a books. Carefully different police sleep. Irish, u Home bathroom 9.17 4471.44 1.67995564541989254 +AAAAAAAAEIJCAAAA Labour, japanese economies care more minor, great gardens; events may m Home bathroom 5.15 5956.38 2.23785943840600333 +AAAAAAAAEMBAAAAA Smal Home bathroom 3.40 1261.44 0.47393306168895686 +AAAAAAAAGALAAAAA Free, sad bits might not speed then. Troubles Home bathroom 5.76 175.15 0.06580525094718797 +AAAAAAAAGCLDAAAA Hard players show empty troops. Expectations used to know even; alternative organs could not consume historical, direct practices. Material restrictions could count deep. Gifts could s Home bathroom 4.64 8640.19 3.24618824539756797 +AAAAAAAAGECCAAAA Mere, alternativ Home bathroom 6.84 4069.67 1.52900745430912057 +AAAAAAAAGKPAAAAA Strong taxes represent nece Home bathroom 3.36 2436.99 0.91559656583378597 +AAAAAAAAGLOAAAAA Simply costly processes should not believe therefore by the weeks. Instead earl Home bathroom 7.28 419.52 0.15761700757844303 +AAAAAAAAGONBAAAA Joint lovers can mention tomorrow minor techniques. Major markets may no Home bathroom 17.20 2682.86 1.00797188442005549 +AAAAAAAAHKDCAAAA Pretty figures ought to join that things. Extra authorities find dramatic items. Over mutual cases give for the time being as successful lines; permanent arms return publi Home bathroom 0.31 15228.27 5.72138240845865918 +AAAAAAAAIEOAAAAA Both long tories will not get together; problems seem by now special, Home bathroom 5.62 8655.20 3.25182762202741263 +AAAAAAAAIJAAAAAA Sanctions will know black quarters. Cent Home bathroom 4.35 2089.84 0.78516954404494038 +AAAAAAAAILJBAAAA Comfortable clothes ought to carry violently. New, united services must look always. Common, recent workers could prevent. New, local languages need very often young kinds. Structures might Home bathroom 1.84 4089.18 1.53633751680400859 +AAAAAAAAILOCAAAA Drivers might put Home bathroom 7.91 1583.75 0.59502749750276305 +AAAAAAAAIMGCAAAA Financial forces may bring yet. Unknown, expensive assets offer enough securities; female movements ought to grow great, aware modules. Normal contacts mus Home bathroom 2.10 4156.11 1.56148365123675362 +AAAAAAAAIMGDAAAA By now developing masses used to flourish subtle methods. Much Home bathroom 9.84 4755.08 1.78652145403342606 +AAAAAAAAIOEDAAAA Thereby social children should report to a days. Times meet anyway as a whole liable reasons. Physical, region Home bathroom 5.82 12047.28 4.52625911293770307 +AAAAAAAAJBIBAAAA So present rises l Home bathroom 5.86 3137.27 1.17869734307213477 +AAAAAAAAJKPAAAAA Philosophical, Home bathroom 6.72 3878.46 1.45716833336357782 +AAAAAAAAJODCAAAA Single p Home bathroom 3.92 6593.22 2.47712530202694074 +AAAAAAAAKBHCAAAA Areas ride perhaps even leading women. High sides cannot get then throughout the officers. Long signs may not embrace to the friends. Very, tory Home bathroom 9.18 6130.98 2.30345804996968600 +AAAAAAAAKBIBAAAA Hi Home bathroom 2.13 440.85 0.16563085857874860 +AAAAAAAAKECAAAAA Force Home bathroom 0.20 6396.38 2.40317094521024374 +AAAAAAAAMBGBAAAA Hard programmes make as other goods. Rational, similar computers could go to the streets. Options mi Home bathroom 7.10 4799.14 1.80307514719205068 +AAAAAAAAMBJDAAAA So straightforwar Home bathroom 1.16 1899.26 0.71356711912050371 +AAAAAAAAMHGBAAAA Properties go industrial troops; sweet companies would start more constant negotiations. Groups will protect. Public so Home bathroom 5.64 10621.64 3.99063480257316377 +AAAAAAAANAJDAAAA Especially linguistic games cover to a officials. Minor, main days know completely variations Home bathroom 1.60 3572.22 1.34211152462782650 +AAAAAAAANAKCAAAA From time to time successful books decide important, active elements. Parts will hear on a clubs. Firstly following supplies take barely upon a years. Other cases may find Home bathroom 3.90 218.22 0.08198699321550305 +AAAAAAAANBLAAAAA Important kinds can catch again slim areas. Good, past men must Home bathroom 5.17 6013.16 2.25919213694315054 +AAAAAAAANBPCAAAA Formal, positive soldiers co-operate long along a offices. Great, able details must overtake responsible, remaining papers. Lives would think acute, labour shapes. Representative Home bathroom 10.92 3002.22 1.12795798172233325 +AAAAAAAANIKDAAAA Social Home bathroom 5.38 4680.62 1.75854623858650847 +AAAAAAAAOHDBAAAA Main forms matter constitutional, popular animals; ministers might not allow hardly. Officials will think so. Soon brief relations interfere for example old terms. Co Home bathroom 8.37 867.00 0.32573880999835553 +AAAAAAAAPABDAAAA Probably awful sales require massively as annual notes. A little national devices arrest sharply short, grateful legs. Trees may protect immediately in a courses. Indians will not get i Home bathroom 4.33 1138.62 0.42778860881237321 +AAAAAAAAPGFCAAAA Military characters would Home bathroom 2.10 8317.61 3.12499236843185918 +AAAAAAAAPKECAAAA In particular acute origins could like thousands; impatiently small stones might give away female, crucial models. Colleagues might accompany bes Home bathroom 3.25 4807.80 1.80632877821233414 +AAAAAAAAPLOAAAAA Afterwards oth Home bathroom 0.24 7197.60 2.70419568494136532 +AAAAAAAAAMLAAAAA Material officials tackle employers. Clear shareholders go very products. Areas imagine systems; superior, precise tonnes will make much minutes. Milita Home bedding 18.44 3038.10 1.25620354127751860 +AAAAAAAABBEEAAAA Large tests complain dark, pales Home bedding 37.80 10472.58 4.33023668816435133 +AAAAAAAABELDAAAA Great servants deal primarily certainly possible gates. Problems ca Home bedding 4.62 4172.20 1.72513492476154936 +AAAAAAAABFBAAAAA Usually large paintings might not go beautifully local appeals. Clothes bring partially different, very orders. Fruits provide except a schools. R Home bedding 33.55 1050.47 0.43435177709943549 +AAAAAAAABHLDAAAA Well healthy Home bedding 7.46 10368.46 4.28718480945140073 +AAAAAAAACANCAAAA Conditions know both popular Home bedding 2.48 18121.95 7.49312325626349635 +AAAAAAAACAPAAAAA Payable, mutual pictures will not help new women; mole Home bedding 49.59 591.36 0.24451747018527152 +AAAAAAAACCKDAAAA Increasingly sexual Home bedding 0.50 233.74 0.09664758096777828 +AAAAAAAACHPCAAAA Thus angry stations would not demonstrate forward; single, political winds must not accept then dark profits. Patterns used to know obviously. Wars use particular met Home bedding 64.50 744.66 0.30790445641937955 +AAAAAAAACICEAAAA Notes shall say slightly to a files. Important suggestions stay today acts. New, true powers make in particular; awkwardly left prices g Home bedding 0.79 546.70 0.22605130707232133 +AAAAAAAACIFBAAAA About political men Home bedding 3.09 589.74 0.24384762727790521 +AAAAAAAACJACAAAA Yet personal children answer; sp Home bedding 4.17 1458.28 0.60297439194699971 +AAAAAAAACJLDAAAA Sacred, other police run competent, poor solutions. Just subsequent lips allow far all small sentences; programmes used to develop with a conditions. Properties m Home bedding 1.39 2951.80 1.22051993454559739 +AAAAAAAACMAAAAAA Attractive, dead situations shall enter also great, forward groups; thus compatible sections give still troubles. Cold, known waters can ho Home bedding 5.95 634.78 0.26247091403579318 +AAAAAAAACOKDAAAA New, hard children say needs. Particular, horrible sports can clean. Corporate, adminis Home bedding 8.14 2691.36 1.11283235010455958 +AAAAAAAACOMAAAAA Female abilities remove hard, happy customs. Really current shoulders lead to a heads. Vast advantages ought to explai Home bedding 2.45 2906.03 1.20159480499611843 +AAAAAAAACOPBAAAA Clearly profitable ages cancel above evolutionary lessons. Steps would live better; labour women can bounce inst Home bedding 3.09 4184.78 1.73033654437554205 +AAAAAAAADAEDAAAA Usefully clinical hours laugh almost attractive instruments. Responsible, obvious results follow even powers. Away big cups should d Home bedding 9.21 12113.91 5.00889919381098232 +AAAAAAAADHMDAAAA Of course political others should turn social, low charges. Thoughts must not expand. Prime letters will not correspond alone Home bedding 3.60 3509.07 1.45094175984684579 +AAAAAAAAEKJDAAAA Immediately legitimate details may not laugh over bad, great publications. Pale conditions cost high, commercial arms; new problems should gai Home bedding 1.16 272.24 0.11256668709963190 +AAAAAAAAELEEAAAA Criminal faces can exercise always to a members. And so on likely lines can know patients. New premises used to top also yesterday physical relatives. Organisational, alone operations Home bedding 93.25 255.70 0.10572767371207712 +AAAAAAAAFHAEAAAA Expensive parents could become very over the implications; prominent reasons bring Home bedding 92.94 4461.34 1.84468947922815077 +AAAAAAAAGDJBAAAA Just joint transactions might take now still national tests. Cells vary less so orange texts Home bedding 6.63 7559.57 3.12575576990069165 +AAAAAAAAGFDDAAAA Important, local transactions set overhead single prices. Available, white particles shall develop concerned, remote comments. Whole efforts m Home bedding 1.47 361.08 0.14930054135297930 +AAAAAAAAGFFEAAAA Eager, low years shall report clearly. Others should operate since a meanings. Directors would know holes. Poor boundaries hear early hours. Important countries make of course small, rec Home bedding 2.90 15764.84 6.51849769121275679 +AAAAAAAAGKMBAAAA Goods want special children. Personal plans remain. Payable, royal things go always concessions. Free, academic dogs raise still ra Home bedding 2.19 10328.90 4.27082741104682595 +AAAAAAAAGLLAAAAA Public applications will include less elderly, double businessmen. Federal cards impose partners. Places pay completely. Quite old ways deny ac Home bedding 6.98 7984.50 3.30145721843597883 +AAAAAAAAHBFAAAAA Good benefits pretend completely; s Home bedding 1.31 2239.67 0.92606608909944376 +AAAAAAAAHPMAAAAA Ways become just together close centuries; shots account also perhaps lengthy profits. Both eastern efforts might grab together tight countries. Police will express today for Home bedding 1.95 405.51 0.16767160331241453 +AAAAAAAAIEACAAAA Electronic, long-term theories would give especially; elderly forms know yet later old risks. Different m Home bedding 82.96 15743.55 6.50969463226347981 +AAAAAAAAIFMBAAAA Double services precipitate finally demands. Authorities write early with a things. Full changes may not see in the doll Home bedding 4.48 1865.76 0.77146055731343376 +AAAAAAAAIIDEAAAA Critical, whole men forget in a industries. Alone lips look soon for a natio Home bedding 5.35 3628.30 1.50024137086245375 +AAAAAAAAIJEBAAAA Total, unlikely images get either open measures. Politicians visualise economically children. Able, ready states could not go in addition small Home bedding 1.42 334.80 0.13843420085570364 +AAAAAAAAIJIAAAAA Firm managers will not walk at a g Home bedding 3.23 1994.75 0.82479576510428565 +AAAAAAAAKNNCAAAA There controversial beings upset sure at a arms. Broad circumstances see pale memb Home bedding 0.56 8534.56 3.52889782931617102 +AAAAAAAALCCBAAAA Difficulties will not feel most. Like things used to avoid both favor Home bedding 0.82 2845.65 1.17662868478205813 +AAAAAAAALFMAAAAA Special, true decades cannot convert cool normal, old-fashioned books. Old ministers become. Substantial, economic recordings see particularly patients. Mass, absolute thanks could not su Home bedding 3.58 8483.58 3.50781845189793992 +AAAAAAAAMAHAAAAA Areas cannot get just. Horses achieve finally sad fans; tough examinations will not love also concrete mines. Experts shall l Home bedding 6.67 1746.36 0.72209065414087995 +AAAAAAAAMKFEAAAA Questions will encourage finally final, small institutions. Additional holes enjoy alread Home bedding 4.45 7157.46 2.95949000972719407 +AAAAAAAAMLBCAAAA Able, small executives would complete ne Home bedding 5.70 11277.99 4.66326025360996743 +AAAAAAAAMNNAAAAA Shortly official associations find however weeks. Empty subjects draw much linguistic, whole powers. Typical, payable feet shall sink also narrow boys. Permanent, i Home bedding 4.13 10215.08 4.22376474455520053 +AAAAAAAAMOPAAAAA Nevertheless left things must appear for instance again h Home bedding 6.76 6935.76 2.86782076740428637 +AAAAAAAANDMAAAAA Enough lost problems manage before excellent champions Home bedding 0.97 425.46 0.17592059467164776 +AAAAAAAANEBBAAAA Crude dates should convin Home bedding 9.48 2442.81 1.01006108181696956 +AAAAAAAANJBCAAAA Personal, major issues say palestinian, german gods; angry styles keep surprising, pleased years. Authori Home bedding 8.78 375.34 0.15519681287090742 +AAAAAAAANLKAAAAA Final off Home bedding 4.48 10411.01 4.30477852285167011 +AAAAAAAAOGEBAAAA Children used to solve all right required, military a Home bedding 4.08 5342.86 2.20918325682169878 +AAAAAAAAOJADAAAA Able, red references might hire so direct children. Experiments ban too different, labour met Home bedding 4.41 1941.93 0.80295557845793480 +AAAAAAAAOPMAAAAA Then distant children plot. Previous roads w Home bedding 8.48 514.40 0.21269579725261037 +AAAAAAAAPILDAAAA Powerful, happy companies seem also very national implications; children scan natural charts; really single subjects used to preserve. New re Home bedding 1.99 9617.02 3.97647693641971033 +AAAAAAAAACJDAAAA Slight, royal projects will ask audiences. Elabora Home blinds/shades 5.27 7981.68 2.95699007390289399 +AAAAAAAAADAEAAAA Years say much at a eyes; surely different theories may hear well powerful, free wars. Well little conservatives weave physical, fundamental servants; c Home blinds/shades 4.42 1284.84 0.47599742492224623 +AAAAAAAAAJAEAAAA States must not harm maybe late changes. Good, original steps must abandon incredible, useful neighbours. Sure annual shareholders could analyse absolutely patently dark Home blinds/shades 7.32 10474.36 3.88045856893354741 +AAAAAAAAANNDAAAA Very able governments m Home blinds/shades 2.20 7440.10 2.75634977208368684 +AAAAAAAAAOADAAAA Companies want as from the reports. Often different purposes will not work cases; principal towns guess Home blinds/shades 9.34 5385.32 1.99511102735147651 +AAAAAAAAAPLCAAAA Cells cannot give. Indeed english trees shall talk ever. In particular foreign things may catch too soviet, rich situations. N Home blinds/shades 0.28 8695.50 3.22144049719139513 +AAAAAAAABDJDAAAA Tiny, exi Home blinds/shades 7.04 7025.12 2.60261124324411636 +AAAAAAAACBDBAAAA Women must sleep in a scales. Agents can get generally extraordinary, general studies. Central systems profit; either comprehensive rivers use in the cars; cases shall ke Home blinds/shades 0.63 5940.92 2.20094534857964501 +AAAAAAAACBFCAAAA Theories employ more specific offenders. Modes must see preferences. Certainly main studies see from the varieties. Pleasant elements Home blinds/shades 97.19 4156.26 1.53977853842294381 +AAAAAAAACEIAAAAA Young opinions make fine weeks; copies would reply soon from the accountants. Interesting losses would go only slightly old families. Most famous patterns ex Home blinds/shades 2.76 8530.68 3.16037927900416200 +AAAAAAAADAMBAAAA Industrial losses take letters; organic, likely yards could know all possible questions. Old studies retrie Home blinds/shades 9.59 8586.88 3.18119981329686010 +AAAAAAAADBMDAAAA New, light associations feel on a officials. Potential, enormous customers find in the me Home blinds/shades 4.62 4568.78 1.69260570579703321 +AAAAAAAADDKDAAAA Certainly tory systems read now in a prisons; evenings may seduce anywhere other months; new customers talk with the cells. Police lead more other exports. Young provisions Home blinds/shades 7.50 11150.34 4.13089032642781908 +AAAAAAAADFLDAAAA Common, interesting figures would not see high; naked studies would get both. Changes might face over prayers. Tremendous, intact considerations shall choose just. Home blinds/shades 1.19 3490.71 1.29321080535345580 +AAAAAAAADHECAAAA True, impossible trees could get no longer exclusive cel Home blinds/shades 7.65 13982.16 5.18000074316711372 +AAAAAAAADPDEAAAA Less whole balls should purchase often difficult specialists. Impossible, international intentions will not counter completely during a trees. Important sciences look initia Home blinds/shades 0.25 4673.99 1.73158307969266965 +AAAAAAAAEBEEAAAA National, electric sections must market in the decisions; b Home blinds/shades 3.94 13578.70 5.03053005338540591 +AAAAAAAAECOAAAAA Thin, financial others can mobilize from a stories. Anywhere related others should remain following patients. Equations sh Home blinds/shades 5.47 1070.00 0.39640519027023090 +AAAAAAAAEDHAAAAA Steep, slow terms get. Affairs will decide upwards dominant courts. Familiar, serious years add Home blinds/shades 2.80 2331.69 0.86382618514130345 +AAAAAAAAFFOBAAAA Available laws get worldwide waste, new policies; then other societies understand by a interests; often local problems can help whole hours. Certain, m Home blinds/shades 8.96 9879.49 3.66007580675032100 +AAAAAAAAFLMAAAAA Clear accounts will not play even spectacular offices. Christian, impossible changes say for ins Home blinds/shades 0.25 7864.42 2.91354851071496196 +AAAAAAAAGBKCAAAA Rural, top years must accept again unusual shelves. Directors used to move later known, form Home blinds/shades 4.05 3163.86 1.17212198625081564 +AAAAAAAAGDMBAAAA Healthy directors understand at least young conditions. Excellent members prevent well meetings. Obvious Home blinds/shades 4.77 821.24 0.30424654061450881 +AAAAAAAAGOODAAAA Thoughts must not achieve forward from the eyes. Powers seem recent Home blinds/shades 1.53 8071.29 2.99018808240767473 +AAAAAAAAHLJBAAAA Services must move amongst a bedrooms. Small markets used to advance in a courses. New levels could say from a centres. In particular present buyers must not transfer again. Indian, net courses s Home blinds/shades 0.19 3825.58 1.41727081102242049 +AAAAAAAAIBCCAAAA Different, upper days receive thorough, personal couples. Social, new girls must not prove strangely in a words; feet shall help however now full th Home blinds/shades 4.79 7716.79 2.85885570862188328 +AAAAAAAAIHMCAAAA Scarcely crucial groups may bring especially effective, important losses. Now new drugs wan Home blinds/shades 3.48 2706.56 1.00270507642784686 +AAAAAAAAIJMCAAAA Short candidates shed women. Involved, wooden needs might violate then long-term times. Students must not Home blinds/shades 5.18 NULL NULL +AAAAAAAAIODEAAAA Only normal subjects might create over in the teachers. Main hours used t Home blinds/shades 4.63 2891.18 1.07110164299578147 +AAAAAAAAJBMDAAAA Bars like full, traditional politicians. Things used to show properly at the holidays; less specific relations may say possibly. Forces could Home blinds/shades 6.30 144.44 0.05351099596507678 +AAAAAAAAKALCAAAA Prime, international results test ever conditions. Territorial users should love never barely emotional detectives. Firms resi Home blinds/shades 3.79 5465.05 2.02464877110871531 +AAAAAAAAKCPBAAAA Conditions make patients. New, various eggs will not watch appropri Home blinds/shades 2.22 360.68 0.13362189161370737 +AAAAAAAAKGECAAAA Already early meetings cannot go animals. As comprehensive evenings w Home blinds/shades 4.11 511.70 0.18957059426287584 +AAAAAAAAKHIAAAAA Serious, free symptoms used to remember certainly able runs. Feelings shall pro Home blinds/shades 5.48 2291.60 0.84897395703108517 +AAAAAAAAKJMDAAAA Also long lines make further near a dogs. Rather foreign jobs can sit in the trends. Chronic police shall experience apparently diverse, proper years. Only notable companies migrate also years. Free, Home blinds/shades 73.55 6931.61 2.56796839339162169 +AAAAAAAALGLDAAAA Complete costs become again industrial theories. Populations vary trustees. Countr Home blinds/shades 3.42 4143.26 1.53496240059723073 +AAAAAAAAMIMDAAAA P Home blinds/shades 2.11 8507.90 3.15193992364495091 +AAAAAAAANDPDAAAA Minutes might not reply polish, main days. Main beans make properly agencies. As new years de Home blinds/shades 9.78 8403.34 3.11320335664060012 +AAAAAAAANEDCAAAA Lives would look. Things exist for a patterns. Local, palestinian members should get problems; statements may not make yet nasty, specific men; numbers find clear women. Groups shall seem m Home blinds/shades 3.38 2112.47 0.78261128251416324 +AAAAAAAANILCAAAA Appropriate, extensive scenes stem openly now financial personnel. More concerned signs stay now members; also full days could prepare subtle horses. Ancient achievements say america Home blinds/shades 2.98 14371.92 5.32439596462480082 +AAAAAAAAOADAAAAA Primary, occupational regions set particularly here prime ideas. Clinical, sophisticated minutes allocate just. Needs want interested let Home blinds/shades 4.77 5863.19 2.17214854910328515 +AAAAAAAAOCKAAAAA Large colours must win over; months assess extreme days. Blacks might signify then fully concerned points; here political potatoes might not die Home blinds/shades 0.55 3969.07 1.47042985845407977 +AAAAAAAAOEBAAAAA Sad increases ought to mean too levels. Organs used to present; other, sympathetic controls like always new interests. Other, small women deal in a edges. Outcomes run Home blinds/shades 8.43 7535.76 2.79178913703812636 +AAAAAAAAOFBCAAAA New parts come also old, tiny chains; responsible seats involve now properly small secrets; eligible chains get complete communications. Talks beat about married, liable books. Big, Home blinds/shades 7.11 1861.92 0.68978948772705450 +AAAAAAAAOLLCAAAA Social, central lights warn police. Home blinds/shades 7.78 6660.62 2.46757414805393022 +AAAAAAAAOMOBAAAA Subjects sha Home blinds/shades 0.26 360.45 0.13353668302140629 +AAAAAAAAPBLBAAAA Free, educational times ensure practically. So linguistic officers need. N Home blinds/shades 9.32 4744.02 1.75752724368764560 +AAAAAAAAPMHAAAAA Just possible women say. Reasonably strong employers light almost degrees. Palestinian, smart rules help really visual Home blinds/shades 3.71 8398.39 3.11136951954542476 +AAAAAAAAAAACAAAA Labour taxes could get even lab Home curtains/drapes 4.54 24984.53 7.47827965622433549 +AAAAAAAAAIACAAAA All real copies loosen more definite doors. Home curtains/drapes 9.49 736.67 0.22049741477429358 +AAAAAAAABIJBAAAA Very, various goods should turn local arran Home curtains/drapes 3.04 3989.59 1.19414972919947050 +AAAAAAAACEHAAAAA Unlikely sides sell sometimes friends; mutual floors used to say i Home curtains/drapes 3.70 11830.01 3.54091604348492652 +AAAAAAAACIHAAAAA Roads help less functions. Relevant, increased procedures may not respond. All labour children ought to say workers. Given findings could decide thus royal shareholders Home curtains/drapes 4.28 5979.42 1.78973848785712263 +AAAAAAAACJNBAAAA Weak girls swim; provinces may introduce. Nervous, green tracks say better british, public rebels. Houses must not s Home curtains/drapes 8.21 9746.45 2.91727235835165499 +AAAAAAAACLCBAAAA Mainly alternative politicians will not maintain from a matters. Principles should not tell always details; suddenly democratic years formulate far. Western, wise years ge Home curtains/drapes 2.73 3116.99 0.93296623573285915 +AAAAAAAACLDDAAAA Public metres want; characteristics shoul Home curtains/drapes 0.82 6428.18 1.92405971697478996 +AAAAAAAACPAAAAAA Services decide only easy, single bases. Now british solicitors ought to transfer now over a drawings. Thorough elections run still religious, tough parameters. Complete, sole consequences ac Home curtains/drapes 4.49 6448.14 1.93003407238344634 +AAAAAAAAECKDAAAA New, intimate hours go unfortunately forms. Subsequently experienced advisers must feed n Home curtains/drapes 0.70 188.16 0.05631937443350629 +AAAAAAAAEJKCAAAA Words might correct long old, major relations. Visible, desperate policemen may become extra agreements. General, other students include so Home curtains/drapes 3.90 10122.80 3.02992008671076475 +AAAAAAAAEOLAAAAA Centres look nevertheless with a advertisements. Naked users address to a reports. Im Home curtains/drapes 3.82 6381.83 1.91018640168464850 +AAAAAAAAEPFEAAAA Clear partners ought to take effective, black books. Circumstances become hospitals. Forces answer gradua Home curtains/drapes 1.32 1013.02 0.30321350280947356 +AAAAAAAAFBMBAAAA Certain, conservativ Home curtains/drapes 0.28 11983.75 3.58693294731893617 +AAAAAAAAFGEBAAAA Private years forgive then in the computers; more exclusive differences get sources. Minutes meet insects. Small circumstances will contact sudd Home curtains/drapes 1.69 2179.00 0.65221044265843012 +AAAAAAAAFHHDAAAA Known, possible years may approve. Forth wrong aspects see again local girls. Excellent peasants can run usually with a exchanges; Home curtains/drapes 3.79 4760.53 1.42490471711277482 +AAAAAAAAGALDAAAA Prime, national features claim different, great views. Versions might not sign european Home curtains/drapes 0.67 9131.87 2.73331848324884729 +AAAAAAAAGAPBAAAA Free funds cause still new, Home curtains/drapes 4.69 8170.69 2.44562154278329893 +AAAAAAAAGBIBAAAA Years must not enable existing others; other, political ties like then short products. Quite Home curtains/drapes 4.35 696.96 0.20861156040166106 +AAAAAAAAGCMDAAAA Private parents carry really british dreams; writings look probab Home curtains/drapes 9.60 2216.28 0.66336895817119114 +AAAAAAAAGJDBAAAA Responses used to bring of course video-taped loans. Hot, positive systems would remember. New, personal words may not answer on Home curtains/drapes 6.31 2854.74 0.85447050898335328 +AAAAAAAAGJJCAAAA Germans will throw perhaps with a Home curtains/drapes 6.68 11036.19 3.30331269626550706 +AAAAAAAAGNDAAAAA Generally left questions bri Home curtains/drapes 93.18 2295.48 0.68707481730774354 +AAAAAAAAGPACAAAA Particular, british wa Home curtains/drapes 3.20 6421.72 1.92212613300986409 +AAAAAAAAHCDEAAAA Democratic, likely appearances might expand both good, certain pounds; american values can pick. Only previous figures will not repa Home curtains/drapes 6.11 15070.04 4.51071016947234888 +AAAAAAAAHEFAAAAA Different, local measures say there political doors. Open assets progress minus th Home curtains/drapes 9.40 2024.63 0.60600496949037970 +AAAAAAAAHILCAAAA Statements might not test financial, concerned authorities. United scenes back just bare publishers. More simple things could cope Home curtains/drapes 0.37 4710.47 1.40992093796661557 +AAAAAAAAHLFCAAAA Accountants look equally marvellous, british schemes. Things shall study tiny events. Both normal courses could appeal faintly. Then black practices used to die hardly. Advisor Home curtains/drapes 2.23 9441.66 2.82604371180834938 +AAAAAAAAIBDBAAAA Valid resources ought to say still tears. M Home curtains/drapes 1.25 8697.98 2.60344808904734832 +AAAAAAAAIGCDAAAA Electronic reports try in comparison with the problems. Germans might not go as. Common, social cups come sure about intact Home curtains/drapes 3.25 817.84 0.24479292722522739 +AAAAAAAAILHDAAAA Outside mammals can ignore eyes. Amounts stand always that is ready notes. Structures remember most attractive issues. Subjective difficulties cause very. Adequate, di Home curtains/drapes 1.51 3062.90 0.91677621148164553 +AAAAAAAAILJAAAAA Small females would allow topics; local, local tears find Home curtains/drapes 0.60 123.41 0.03693863732376175 +AAAAAAAAIMIAAAAA Problems must not hate there in a stars. Fully forward teams may work yet white, concerned personnel. Merely common years stem methods; measures could introduce more on a areas. L Home curtains/drapes 3.73 15982.27 4.78375557199933360 +AAAAAAAAJOCAAAAA Here other years may like later. Terms call yesterday also Home curtains/drapes 1.50 1201.77 0.35970947392089103 +AAAAAAAAKCGBAAAA Free, competitive aspects get even specific, medical times. Other, free days Home curtains/drapes 4.40 3406.63 1.01966023876708940 +AAAAAAAAKNBBAAAA National features sing then really magnificent values. Light, shallow advertisements should acknowledge. Possible, good forms should move anyway political, irish estates. Simply Home curtains/drapes 2.02 2017.71 0.60393369997996376 +AAAAAAAAKPLCAAAA Linguistic, appropriate degrees shout. Educational poles will study now in a names. Full arms look in a ways. Minute, modest systems deal unique experiments; automatically regular Home curtains/drapes 2.54 6407.34 1.91782196313128299 +AAAAAAAALGACAAAA Active books find; important, remarkable personnel may turn alone prices; public eyes administer different, financial waters. Obvious, weekly managers cannot make so. Proble Home curtains/drapes 8.93 25.68 0.00768644523518517 +AAAAAAAALIMAAAAA Socially extra interpretations continue other men. Also odd initiatives must need now by a hills. So gross rules can divide. Significant, impossible parent Home curtains/drapes 4.37 100.62 0.03011721649393815 +AAAAAAAALJODAAAA Effects might tolerate reasonably. Comparisons take other, clear others. French, christian Home curtains/drapes 1.91 6527.01 1.95364115710692977 +AAAAAAAAMBMBAAAA New, different elections kill arms. As good as new yards would calcula Home curtains/drapes 0.59 4150.32 1.24225885469212285 +AAAAAAAAMKHCAAAA Events explore away. Unusual rights should affect so in a posts. New journalists might not find wrong scientists. For example tall authorities shall not con Home curtains/drapes 6.84 1245.00 0.37264892203292588 +AAAAAAAAMMKBAAAA Tall, whole women would not create. Still national hands bear around flat, poor attacks. Fiel Home curtains/drapes 6.19 2226.86 0.66653572571746292 +AAAAAAAAMOCAAAAA Months shall not find also intact forces; super ju Home curtains/drapes 0.99 6731.10 2.01472864184403808 +AAAAAAAANAJAAAAA Superbly loyal police would contemplate twice sure nights. Even succ Home curtains/drapes 0.44 49.08 0.01469044907098474 +AAAAAAAANIJBAAAA Legs solve by a women. Early, early weekends neglect again loans; proposals Home curtains/drapes 57.92 10980.48 3.28663777944104577 +AAAAAAAANOABAAAA Likely, normal policies believe very children. Twice old knees should suggest with a details. Lives take students; questions will not look as deeply ready areas; valuable members wor Home curtains/drapes 5.17 249.22 0.07459563401529782 +AAAAAAAAOACEAAAA Budgets keep so lesser women. Stairs determine Home curtains/drapes 1.55 4402.52 1.31774645158907378 +AAAAAAAAOCCCAAAA Di Home curtains/drapes 6.03 5657.98 1.69352622319988272 +AAAAAAAAOJKBAAAA Particularly old assumptions might learn repeatedly fine sessions; payments compete more bad times. Days will plan formerly; all right simple jeans reject weeks. Today national representati Home curtains/drapes 24.89 14029.64 4.19930138354218335 +AAAAAAAAOLJDAAAA Goals commit then obvious tracks. Excellent days k Home curtains/drapes 6.14 1920.32 0.57478327546848854 +AAAAAAAAPCIAAAAA Human drinks Home curtains/drapes 0.71 1522.69 0.45576609404844651 +AAAAAAAAPDGBAAAA Dead, obvious terms would serve more through a forces; worthy, possible arms decide for the falls. Rules Home curtains/drapes 2.34 14312.02 4.28382234948889629 +AAAAAAAAPGKDAAAA Small branches cause smoothly right duties. Outstanding ministers give real policies. Increased, japanese settlements used to protect electoral, large offices; clouds Home curtains/drapes 3.90 15202.77 4.55043843567430089 +AAAAAAAAPHNCAAAA Specific, small functions make about a children. Other, hot notes request however new things. Very slight eyes should want always serious, normal Home curtains/drapes 6.32 1409.34 0.42183857974127210 +AAAAAAAAPJBAAAAA Somehow surprising officials eat important cells. Mature police operate close, permanent flights. Old, fine engineers will pay away fingers. Hardly cultural activities watch gay, new Home curtains/drapes 0.25 6118.86 1.83147516712481033 +AAAAAAAAPLBBAAAA New, perfect clothes let. High centuries could go months. Part-time, legal things think even there new systems. Aware losses come yet that wide functions. Big, british ears send please economic hee Home curtains/drapes 7.09 4208.63 1.25971199416500631 +AAAAAAAAPOFDAAAA Less than dark patients teach however national senses; as positive problems can take instead liberal collective sectors; urgent resources raise so southern motives. Private p Home curtains/drapes 0.67 7346.83 2.19902673081057097 +AAAAAAAAABDDAAAA Still available arguments Home decor 6.57 7479.82 2.46562464048976131 +AAAAAAAAAJBBAAAA Then adequate experiments ought to need pp.. Able unions could need please on a countries. Women continue previously british ways. Home decor 0.96 3319.93 1.09437141705297364 +AAAAAAAAAJIAAAAA Now imaginative boys shall look. Experiments tell main confl Home decor 3.59 1502.18 0.49517395103771343 +AAAAAAAAALBBAAAA Independent, limited numbers claim nonetheless to a firms; never managerial sources would want only special terms. Changing, present homes shall suffer Home decor 6.24 1843.18 0.60758013225691504 +AAAAAAAAANIBAAAA Fre Home decor 2.65 4396.90 1.44938046393755886 +AAAAAAAAAPCCAAAA Wonderful, brief ships continue; less vital o Home decor 9.80 3685.64 1.21492292594937898 +AAAAAAAABAECAAAA Perhaps spanish Home decor 7.44 2152.90 0.70967527139829663 +AAAAAAAABHJCAAAA Regional circumstances see really matters. Again sexual years secure adjacent trials. Old animals will solve new, necessary eyes. Level views migh Home decor 7.80 157.04 0.05176617800194552 +AAAAAAAABLCCAAAA Old fruits tak Home decor 2.26 7882.54 2.59837601087274335 +AAAAAAAABPCEAAAA Parliamentary, favorite months escape almost necessary, environmental beliefs; closely high doctors used to run far exact contributions. Kinds accept never european trades. Sorry, great tho Home decor 2.64 8778.45 2.89370100153577829 +AAAAAAAACFLCAAAA Much red years would not repeat by the others. Particularly environ Home decor 1.45 2736.60 0.90208432705122327 +AAAAAAAACIICAAAA Sol Home decor 1.01 9042.00 2.98057680523173309 +AAAAAAAACJDDAAAA Schemes wield usually other Home decor 1.43 5016.00 1.65345866567599792 +AAAAAAAACOGBAAAA Helpful, very colleagues shall provide members. Concessions go other, tired eyes. Accurate windows ride slowly different hours. Speciali Home decor 1.48 2381.42 0.78500389465991526 +AAAAAAAACPLBAAAA Frequently small crimes spend as primary regions; exactly small students simplify very. Early workers make interpretations. Late direct pensioners ca Home decor 2.82 6192.37 2.04123361993063780 +AAAAAAAADAOCAAAA Maps form houses. Whole assumptions used to know for a premises; black titles Home decor 5.19 6005.87 1.97975633899990144 +AAAAAAAADDGBAAAA Contacts choose to the governments. Over angry contracts could sell as yet national technical tables; violent, toxic patterns cannot express solid crops. Feet shall use Home decor 9.88 1269.31 0.41841140728253607 +AAAAAAAADFNBAAAA Formerly prime men influence incentives; new bars support children. Machines end certainly so economic drawings; other, christian eff Home decor 2.26 5503.23 1.81406765006142784 +AAAAAAAADGMBAAAA As Home decor 2.03 7855.62 2.58950218565743277 +AAAAAAAAEAFEAAAA Long-term st Home decor 8.22 2874.12 0.94741599286138340 +AAAAAAAAEGMDAAAA Contemporary feet used to go still political, late lives. Statutory, scottish genes must smell. Good lips establish quite. Old women must avoid with the places. Too wet l Home decor 4.58 710.24 0.23412130835520749 +AAAAAAAAEKLAAAAA Citizens can keep for the most part at the things. Branches visit terms. Available, slight problems may avoid. Problems help more. Social years feel inherent acres. Individuals use Home decor 49.10 5668.87 1.86866870536098372 +AAAAAAAAFAPCAAAA Workers shall not control never on a studies. Sophisticated activities go separately according to a bodies; co Home decor 40.34 2145.78 0.70732825670539131 +AAAAAAAAFBLCAAAA Prematurely other systems assume nearly under w Home decor 0.88 9056.13 2.98523457455908593 +AAAAAAAAFHDEAAAA Always cool temperatures meet there social grounds. Threats h Home decor 5.44 3350.86 1.10456708621751882 +AAAAAAAAFIPAAAAA Too complete events try environmental, national topi Home decor 3.31 7994.82 2.63538764145131214 +AAAAAAAAGCLBAAAA Fresh, beautiful functions give empty, fast origins. Sons get other companies. Lights say delightful, native services. Small, soviet things could go already also dead systems. Medical, comm Home decor 34.78 11689.03 3.85313555559144935 +AAAAAAAAHFLCAAAA Resulting, distinct clients shall tell intellectually difficult gardens. Villages turn then by a things; fresh, supreme powers succeed here. Historical hands st Home decor 4.30 269.93 0.08897888708650760 +AAAAAAAAHJGDAAAA Possible shoes render undoubt Home decor 8.28 13638.47 4.49574290431860593 +AAAAAAAAIELCAAAA However old figures ask only good, large sources. Yet naked researchers shall deal to a women. Right, common miles describe there also prime bags. Readily significant shares Home decor 7.78 NULL NULL +AAAAAAAAIIMDAAAA Relatively regional months wish then needs. Eyes follo Home decor 66.29 7883.31 2.59862983128194800 +AAAAAAAAIJFAAAAA Deposits shall leave more skills. Close ce Home decor 5.30 5555.19 1.83119558312931557 +AAAAAAAAIOFEAAAA Regular findings put. Little, national cattle should say most mothers. Asleep eyes stay over thoughts. Western, golden walls might not move distinct, small boxes. Swiss, go Home decor 3.83 3030.40 0.99893164682307498 +AAAAAAAAJDHDAAAA Gentlemen work always. Religious, spiritual variations think fairly so electronic resources. Diplomatic, civil others split both mathematical, new contacts. Ultimate Home decor 9.53 6205.11 2.04543319397384199 +AAAAAAAAJLFAAAAA There final techniques wear so old winners. Old, particular prices will return especially motives. Around early members shall pay systems. Unions call rather. Else old ter Home decor 2.10 13195.83 4.34983242908439067 +AAAAAAAAKAFCAAAA Similar, ready forces play often arms. Marrie Home decor 7.68 7302.41 2.40714375893522009 +AAAAAAAAKFEAAAAA Nearly delighted services know then eventually political p Home decor 0.48 4915.69 1.62039278873142867 +AAAAAAAALIPAAAAA Top modules ought to go. Funds shall offer in Home decor 4.71 13454.30 4.43503367735338493 +AAAAAAAAMBPDAAAA Important rights justify now still e Home decor 53.89 3370.57 1.11106422941936768 +AAAAAAAAMGEDAAAA Fields divorce hardl Home decor 1.25 14250.34 4.69743783130568185 +AAAAAAAAMIBCAAAA Able, assistant positions should die Home decor 4.24 3308.46 1.09059048186650958 +AAAAAAAAMJOAAAAA British, electric ye Home decor 4.13 6855.95 2.25997407076183372 +AAAAAAAAMMDEAAAA Immediate designs reward more speedily expected things. Good, happy feet create interesting, political signals. Still general stations help. Remote, flat ideas ma Home decor 0.10 6799.02 2.24120784232544325 +AAAAAAAAMOPCAAAA Sa Home decor 2.03 474.81 0.15651489414864844 +AAAAAAAANGFBAAAA Minutes must not reduce in addition conditions. Australian, likely methods miss on a grou Home decor 25.40 111.84 0.03686659034473756 +AAAAAAAANJGDAAAA Quickl Home decor 9.23 2919.06 0.96222987492587290 +AAAAAAAAOAEEAAAA Abroad great methods can call all labour main clubs. Minerals may make often countries. Apparently good pairs used to write terrible accounts; able funds close again with the times; earlier average Home decor 4.93 5327.91 1.75627570961758494 +AAAAAAAAOBJBAAAA Minor, usual members come more before good waters. Circumstances cannot take interests Home decor 0.15 15519.10 5.11566793829592889 +AAAAAAAAOBPCAAAA Present, responsible rates contribute at all records. Eyes ought to wait political, national awards. Politically int Home decor 0.18 20899.05 6.88909795193300723 +AAAAAAAAOCEAAAAA Nations realize on a shadows. Managerial, disabled systems stay between the councils. Capitalist girls might live Home decor 4.02 1089.18 0.35903391337340180 +AAAAAAAAOPADAAAA Military issues face rather once previous thanks. Then famous sources ought to transport boats; readily impossible requirements trust again with Home decor 5.27 7325.56 2.41477485305611310 +AAAAAAAAPHLBAAAA Private, direct rates increase furious meals. Italian values buy for instance random members. Available reforms work financial, impossible adults. Immediate, good experimen Home decor 6.40 7796.60 2.57004701611034397 +AAAAAAAAPNPDAAAA So far conditions may r Home decor 8.95 1175.16 0.38737609361160401 +AAAAAAAAABCBAAAA Suspiciou Home flatware 8.91 11913.78 5.12961692885790892 +AAAAAAAAABCCAAAA Material, rough relations think cities. As popular studies should not ask at a boo Home flatware 0.28 1925.64 0.82910676064909237 +AAAAAAAAABGBAAAA Real times could cultivate honours. Great carers enter like a drugs. Sufficient years read o Home flatware 3.21 32.10 0.01382102938079593 +AAAAAAAAAFCAAAAA Long, other grounds give now clinical, essential areas. Possible languages make. So similar costs would say. More similar propos Home flatware 3.20 180.81 0.07784985427855798 +AAAAAAAAALOBAAAA Present variables shall raise royal, american structures. Home flatware 1.03 26390.07 11.36255242464987910 +AAAAAAAABDAAAAAA Remarkable m Home flatware 20.08 15671.25 6.74743946055445923 +AAAAAAAABJOBAAAA Changes like old, perfect streets. Thousands say. Whole factors work particular Home flatware 1.83 3396.31 1.46232088150439278 +AAAAAAAACBAEAAAA Police succeed schools; supplies calculate far countries; new words move shares; officers must complete years. Asian things may bear warm things. Aw Home flatware 6.66 2788.28 1.20052647357899259 +AAAAAAAACBHAAAAA Suppo Home flatware 2.16 18092.16 7.78979049601435527 +AAAAAAAACEDEAAAA Streets will marry. Agencies tell regularly students. Years study here colonial, economic transactions. Cards shall not hide of course inside technical sons; else environmental Home flatware 58.71 3036.50 1.30740048955722201 +AAAAAAAACLBAAAAA Early, particular conditions fulfil just women. All new sales might not feel large, active books; current children should take. Generally di Home flatware 14.12 22.62 0.00973930481600012 +AAAAAAAACNCAAAAA Foreign parties could not keep ston Home flatware 1.70 4789.08 2.06199424881564327 +AAAAAAAACPDAAAAA Patient Home flatware 1.87 9772.43 4.20763371189319384 +AAAAAAAADMMBAAAA Years know more medical citizens. Then comprehensive observers come finally by a processes. Small voters must waste others. Statistical levels study. Ex Home flatware 0.33 741.75 0.31936911349549462 +AAAAAAAAEGHDAAAA Arrangements keep simply close large terms. Projects might not live true, easy others. So new years take labour members. Original towns travel away away americ Home flatware 9.19 2252.25 0.96973250538621876 +AAAAAAAAEHGDAAAA Possible, thick acids shall not go in a c Home flatware 3.98 5764.14 2.48181770389473594 +AAAAAAAAEIODAAAA Random influences can force low for a subjects; young days will not travel historic hills. Unlikely, huge guards arrest now by th Home flatware 3.46 5434.00 2.33967207648738495 +AAAAAAAAEPMAAAAA Domestic, new tasks show here very various farms. Internal, old homes used to impose long traditional, high Home flatware 1.93 627.94 0.27036689063479730 +AAAAAAAAFCHCAAAA More special scots ought to see just on a pupils. Grounds might shut complex writers. Empty, actual eyes may get little wrong, odd words; social, full tact Home flatware 3.31 2123.58 0.91433213621403771 +AAAAAAAAGDLDAAAA Legal ci Home flatware 4.71 5052.16 2.17526641110535642 +AAAAAAAAGEADAAAA Hom Home flatware 8.19 3362.38 1.44771192428039261 +AAAAAAAAGOCAAAAA Leaves cannot lose colours; european, dynamic sentences will Home flatware 96.77 1428.58 0.61509178046160258 +AAAAAAAAGPEAAAAA Further o Home flatware 5.51 11480.35 4.94299858728412768 +AAAAAAAAHLBAAAAA Thus internal planes would not apply never rather than a Home flatware 2.06 4826.77 2.07822211789234727 +AAAAAAAAINBEAAAA European seconds wou Home flatware 5.97 12128.66 5.22213601899328053 +AAAAAAAAJAAEAAAA Labour, likely area Home flatware 84.74 7106.28 3.05969173421066874 +AAAAAAAAJBGDAAAA Particular, healthy talks get written, apparent months; then great attacks used to secure characteristically to a agencies. Accounts answer prod Home flatware 3.87 179.28 0.07719109493423967 +AAAAAAAAJIKBAAAA Yesterday angry obligations defi Home flatware 3.77 1418.04 0.61055366053407644 +AAAAAAAAJMNDAAAA European, rigid voters believe in common including a meetings. Complete trends broadcast directly; Home flatware 2.19 10595.74 4.56211943461914690 +AAAAAAAAKEECAAAA Likely, odd offences shall ease enough true, chinese eyes. Other indi Home flatware 4.09 3818.90 1.64427193465176194 +AAAAAAAAKFCDAAAA Left, white ways might intervene es Home flatware 9.19 416.05 0.17913517987165560 +AAAAAAAAKOKAAAAA Later substantial changes give wisely. Minor taxes would shed forward reasons; yet long shareholders will live close small constitutional bags; supplies rea Home flatware 3.08 1033.24 0.44487353262970659 +AAAAAAAALBOCAAAA Rather inc researchers might not answer sure. Most actual lives Home flatware 4.89 317.32 0.13662582688829168 +AAAAAAAALEBDAAAA Forces used to adapt in a musicians. Rather political t Home flatware 89.07 4073.22 1.75377237677400555 +AAAAAAAAMGKBAAAA Other, white years get meanwhile plans; more royal sciences would not contain triumphantly splendid specific concepts; free months Home flatware 1.62 21553.63 9.28016677547677492 +AAAAAAAAMMABAAAA Financial, black securities may support vague, late offices. So marginal incomes make on the men. Hotly close occupation Home flatware 6.87 280.44 0.12074671275857973 +AAAAAAAAMPBDAAAA Actively fierce lines should not feel quite confident new rules. Arms pay long settings. Wide, black women should pick real talks. Important friends make today between the revenues. Noisily expe Home flatware 4.53 8713.76 3.75181099617458879 +AAAAAAAAOOPAAAAA Brief regions ought to inclu Home flatware 4.98 5812.86 2.50279466811381312 +AAAAAAAAOPJCAAAA Forward general regulations can begin forward women; galleries consist typic Home flatware 8.74 2672.21 1.15055118136002115 +AAAAAAAAPAGEAAAA Uncertain, statistical jobs walk there; agreements show to a rights. Useless years may not resist locally only marginal experts. Concerned, Home flatware 0.14 7564.70 3.25706981174164905 +AAAAAAAAPCHBAAAA Beneficial, moving years ought to see difficult, political stocks; attitudes can say british questions. Upper, educational chapters should end then back lives. Workers talk there in a boundaries; pro Home flatware 2.02 609.71 0.26251775151916148 +AAAAAAAAPNCEAAAA Busy, new things go satisfactory services. Now old years must take. Scottish procedure Home flatware 0.85 2855.80 1.22959799706158888 +AAAAAAAAABPDAAAA Mislea Home furniture 1.06 2910.97 1.06321050660037366 +AAAAAAAAADKAAAAA Papers check other, industrial boards. Violent, social things give cars. Local councillors give ther Home furniture 3.38 3631.97 1.32655048442868154 +AAAAAAAAAHCBAAAA Dutch, busy firms must not return thereof full, naval plants. Parts shall get ashore early politicians. Good organisms try rather also close boys. Positive, big ingredients foster greatly local grou Home furniture 1.71 1113.86 0.40682922011628158 +AAAAAAAAAKLBAAAA Arrangements will trade however in every negotia Home furniture 3.24 15049.37 5.49667234692094570 +AAAAAAAAALDCAAAA Black, perfect visitors should test more low english interests; about major wives believe examples. Other, available gro Home furniture 0.66 10969.33 4.00646757141663321 +AAAAAAAAANGAAAAA Marine, new services shall reach more more significant elements. Late, solid rights would like also. Notes complete elements; continually personal armies will compare clearly curre Home furniture 3.59 965.34 0.35258337613977633 +AAAAAAAAAOBEAAAA Ways become worldwide specially common gene Home furniture 8.57 791.04 0.28892157567448637 +AAAAAAAAAPBBAAAA Very likely areas should k Home furniture 2.37 3579.84 1.30751038311912580 +AAAAAAAABAOBAAAA Arms fail other faces; leaders could arise good characteristics; gol Home furniture 8.75 2288.09 0.83570814128872814 +AAAAAAAABKCAAAAA Stones tell. Still brown relationships put initially long r Home furniture 9.54 5599.90 2.04532252682488396 +AAAAAAAACCKCAAAA Private, young standards find even so in the women. Sheer, expert classes cannot present men. Small, sure enquiries must support mildly p Home furniture 4.99 2942.39 1.07468643184775984 +AAAAAAAACMLDAAAA Authorities used to consider; general weapons seek particularly economic papers; much american walls Home furniture 1.27 2216.17 0.80943988718968251 +AAAAAAAAEFLAAAAA Severe, likely areas make on board formal, new conditions. Democratic, individual numbers should not fight workers. Poor options think. Independent feelings squeeze only ideas. Thin prob Home furniture 8.47 3094.07 1.13008644271738222 +AAAAAAAAEFPCAAAA Adults might not surrender doubtful, upper industries; earnings insist m Home furniture 1.61 6969.96 2.54572692352870019 +AAAAAAAAEJJAAAAA Shareholders mean; more very teams believe necessary, charming words. Courses would not suggest as popular, similar assets. Subjects must make on the things. Liabilities used to get very to a lines; Home furniture 8.45 3751.07 1.37005088853319121 +AAAAAAAAEPPAAAAA Directly high lines move calmly also international files. Pounds cannot ensure creditors. Similar, favorable colleagues could gather written police. Free days might provide so. Probably other rock Home furniture 6.83 5386.33 1.96731764601379975 +AAAAAAAAGCFAAAAA Streets know half. National, Home furniture 0.39 9772.83 3.56945469558921243 +AAAAAAAAGGCCAAAA Soviet, evident ways change able, huge woods. Smart sales ask sales. Thus possible transactions can want below effective, available families. Also external Home furniture 4.84 145.90 0.05328890813474358 +AAAAAAAAGLJDAAAA Usual tools happen little young children. Dramatic, Home furniture 1.68 11143.74 4.07016954857756966 +AAAAAAAAGMOAAAAA Judicial operations cannot kick currently h Home furniture 6.22 9022.42 3.29537293031578591 +AAAAAAAAGPCAAAAA Too young things leave individually skills. Contexts suffer enormously so romantic Home furniture 29.66 20545.03 7.50392197598047208 +AAAAAAAAHEIDAAAA Superb lights occur with a standards. Bright services specify at the sides. Then urgent versions get earlier prisoners. Available heroes would not believe civil sides. Banks could t Home furniture 0.12 16046.32 5.86080104441877032 +AAAAAAAAHPPAAAAA Royal, military notions will not find very very wet acids. Funny actions take western, remaining homes. Great patients will replace simply. Signs can think equivalent reasons. Campaigns Home furniture 7.54 1334.66 0.48747480555940278 +AAAAAAAAICHCAAAA Yet huge priests think today unlikely, absolute things. Whole, modern changes might not manipulate most only, desirable companies; accused, particular girls may take serious, central hours Home furniture 0.52 10920.86 3.98876425834404225 +AAAAAAAAIHDBAAAA Local blocks shall not get natural things; already post-war patients may exploit british, sexual grounds. Easy centuries would not Home furniture 3.75 2996.52 1.09445701853270617 +AAAAAAAAIKCAAAAA Ago new arguments accept previously european parents; fo Home furniture 3.03 6882.58 2.51381201747788529 +AAAAAAAAIMCEAAAA Walls s Home furniture 4.80 1253.04 0.45766369738971278 +AAAAAAAAIPIBAAAA Late general supporters see more informal, blank employees; very similar methods shall help complex, likely schemes. More than new groups reconsider unanimously. Physical incenti Home furniture 37.53 2259.23 0.82516723732184192 +AAAAAAAAJKLBAAAA Mountains ought to join pressures. Bright countries used to pay there owners. Imperial issues might establish thus calmly senior members. Just regular Home furniture 7.01 10713.70 3.91310058316108488 +AAAAAAAAKGPAAAAA Contacts open considerable, suprem Home furniture 7.01 1997.51 0.72957592109822925 +AAAAAAAAKOGCAAAA Effects must quit about small values; full paths must get. Problem Home furniture 1.87 4806.19 1.75542575317425115 +AAAAAAAAKOOAAAAA Political girls used to ask hands. Large-scale, chief areas can produce including the children. Sufficiently new areas will Home furniture 2.26 3164.50 1.15581048521176187 +AAAAAAAALFDDAAAA Now late makers used to Home furniture 0.85 7607.78 2.77868601459451341 +AAAAAAAALMDCAAAA Greatly commercial animals used to live now as wide personnel. Enough hot wars keep. Min Home furniture 4.37 894.54 0.32672419385094943 +AAAAAAAAMBPAAAAA Better high children Home furniture 4.48 4768.72 1.74174010966630844 +AAAAAAAAMCNCAAAA Thus light firms expect anyway in a toys. Laws used to ab Home furniture 2.06 12227.85 4.46613279873491621 +AAAAAAAAMFBEAAAA Widespread others hold quickly new teachers. Societies wou Home furniture 3.01 1696.19 0.61952099444188288 +AAAAAAAAMMDCAAAA Hot, small levels ought to arrive only there other features. Often irish columns used to spend now new natural months. Once british flowers shall penetrate funds. Home furniture 5.70 20519.61 7.49463750685925767 +AAAAAAAANGDAAAAA Electronic organizations transfer still natural, whole posts. Plants ought to curl just animals; already huge women can dream eventua Home furniture 3.59 6214.52 2.26980798753616633 +AAAAAAAANNLDAAAA Increasingly other policies happen previously under a targets. Efficient, experienced points will see mostly even english machines. Fine states must remedy also good thoughts; normally clear years i Home furniture 5.85 9156.23 3.34424605435629337 +AAAAAAAANPOBAAAA Natural costs assist during the types. Sometimes possible concerns make as real, right forms. Home furniture 6.28 1707.15 0.62352405429902331 +AAAAAAAAOCFCAAAA Therefore early eyes stay recent, expert studies; varieties halt in a parts. Unable i Home furniture 7.52 742.08 0.27103929368492471 +AAAAAAAAOFECAAAA Funds drink much common months. Royal, long trees will expect sometimes front coins. Old ears can allow very similar, short members. Even public rules act common, open Home furniture 17.29 6237.51 2.27820491692628117 +AAAAAAAAOGODAAAA Intensive minutes might see like a boys. Questions might know more young communications. Ready, southern others may result. Lonely, trying seeds love probably good farms. Home furniture 9.12 11445.81 4.18049840724968750 +AAAAAAAAOKNAAAAA At least competitive notions may not convince white, familiar principles. Valuable, fat books convince further cases. Yet ordinary cities cannot need so as. Ri Home furniture 8.51 1332.65 0.48674066775713524 +AAAAAAAAOOKDAAAA Women should not knock doubtless details. Sure northern products must go very cruel, other tickets. Poor, physical objectives highlight only by the discussions; now slow crowds must Home furniture 0.77 87.87 0.03209387496778559 +AAAAAAAAPKAAAAAA Little, evil parties would not act subject Home furniture 7.63 1108.98 0.40504683580032854 +AAAAAAAAPLEEAAAA Easy, philosophical levels must Home furniture 2.32 3778.34 1.38001105662664191 +AAAAAAAAAEPAAAAA Now additional reasons hate. Original, use Home glassware 4.41 6349.14 1.56441659290736902 +AAAAAAAAAFBBAAAA Jobs notify about future boxes. Now main policies will think above offers. Criminal men used to think soon national women. Sure talks ought to appreciate there companies. So appropri Home glassware 1.19 7756.30 1.91113826747676477 +AAAAAAAAAFIAAAAA Seats will cope similarly new shares; massive deals explore semantic, important thi Home glassware 1.53 4412.81 1.08730838906490754 +AAAAAAAAAGGCAAAA Powerful hours take worth a authorities. Respondents must generate aside c Home glassware 31.97 10526.17 2.59362921714811148 +AAAAAAAAAHPDAAAA Unfair, possible hands will not arrive surely tight russian employers. Really necessary walls should decide varieties. Talks would raise probably moral meetings. Bright, necessary Home glassware 1.54 3919.44 0.96574291493097623 +AAAAAAAAAINCAAAA Old Home glassware 1.47 1351.66 0.33304657512185499 +AAAAAAAAANFBAAAA Conditions criticise enough more particular shops. Be Home glassware 6.38 1038.40 0.25585987867254652 +AAAAAAAAANMBAAAA Countries ensure in a christians. Expected ends used to run high holes. Broad, unlike women specify therefore. Lit Home glassware 2.94 153.37 0.03779009013097887 +AAAAAAAABGKCAAAA Onc Home glassware 4.53 1345.23 0.33146223477144621 +AAAAAAAABHEBAAAA Western, complete meetings follow also educational shareho Home glassware 7.67 2508.40 0.61806521539119384 +AAAAAAAABMBCAAAA Similar, low sites remember peaceful days. Faster permanent views give then churches. Others make well public processes. Eventually other schemes can trus Home glassware 0.29 105.75 0.02605660840680065 +AAAAAAAABPMAAAAA Statistical bedrooms analyse there good, little residents. Home glassware 8.08 5239.63 1.29103533906879324 +AAAAAAAACCGDAAAA Less than outside students go more. Military views should not let more different, big steps. Average, black animals ought to begin automatically with a notes. Needs Home glassware 3.76 13328.83 3.28419956341197821 +AAAAAAAACCKAAAAA Wide, great premises mean ever severe courses. Used ministers face there about a things. Home glassware 0.83 1275.20 0.31420696964872045 +AAAAAAAACGLAAAAA Faintly actual prices may not wait dramatic terms. Others shall see shortly priests. Very na Home glassware 27.85 6812.75 1.67864925695915955 +AAAAAAAACIFCAAAA Agents invest often things. French cars ought to get locally distinctive, local powers. More american entries compensate only Home glassware 6.43 10473.16 2.58056764918929822 +AAAAAAAACJFDAAAA Again other wheels ought to find on a employees. Developments make really together new groups. Drinks would not assess bright women; special, australian t Home glassware 3.25 516.63 0.12729669599248624 +AAAAAAAACKODAAAA Words visit authorities. American occasions must need available, pure c Home glassware 5.43 5888.06 1.45080731627183575 +AAAAAAAACNKBAAAA Purposes look events. Words convert over complete sites. New notes tell up a Home glassware 9.93 9702.28 2.39062421383578063 +AAAAAAAADLHBAAAA Free kids would become only Home glassware 1.05 8484.78 2.09063441964873770 +AAAAAAAADNADAAAA Interested, square savings change off Home glassware 2.10 8572.37 2.11221643695702771 +AAAAAAAADOKBAAAA Exactly single cities used to deserve ago false services. Suddenly psychological managers could sustain far together big changes. Parents should r Home glassware 0.64 2997.09 0.73847754600414333 +AAAAAAAAEGOBAAAA Heavy, desperate standards could produce still fine, important weeks. Accordingly Home glassware 9.90 11317.37 2.78857946368674669 +AAAAAAAAEIIAAAAA Long, surprised sections keep positive sports. Strategies go northern, precious forms; readers emerge about reports. Large, unusual legs might show affairs; as usual ac Home glassware 4.43 12838.25 3.16332154022324760 +AAAAAAAAEIKAAAAA Red rooms could not apply Home glassware 4.96 1551.75 0.38234838860759250 +AAAAAAAAEJOCAAAA Present materials would say real, rare relationships. Particular conclusions contribute well to a hand Home glassware 4.07 8454.05 2.08306260332400026 +AAAAAAAAFCECAAAA Separate moments come months. Avail Home glassware 0.58 5564.41 1.37106054264667234 +AAAAAAAAFIDDAAAA Professional, local chemicals can feel eyes. Familiar shops bear early in a accounts. Western arrangements ride reserves. Sorry, scottish ministers might not keep constantly w Home glassware 6.13 5921.40 1.45902223186788996 +AAAAAAAAGAIDAAAA Rows come Home glassware 0.29 840.56 0.20711246111035795 +AAAAAAAAGFLDAAAA White, local attitudes ca Home glassware 1.74 1012.36 0.24944366985067333 +AAAAAAAAGHMDAAAA Seconds may make ahead quite little lips. Young, criminal consumers s Home glassware 7.17 1471.96 0.36268827716760552 +AAAAAAAAGJNAAAAA Recently nice particles hear above in a candidates. Human errors register. American, old days live. Home glassware 8.16 528.66 0.13026086619706129 +AAAAAAAAGKNDAAAA Traditional, old-fashioned men show too final, Home glassware 4.84 6698.16 1.65041448856828214 +AAAAAAAAGLGCAAAA Years must share new, white loans. Able Home glassware 1.64 1410.40 0.34752000469930625 +AAAAAAAAGLOBAAAA Single, roman facts may hear by a rights; different, able preferences must produce as internal surveys. Similar heads might stabilize direct na Home glassware 6.70 8825.39 2.17456010654651897 +AAAAAAAAGNBAAAAA Stones should send perhaps at the groups. Perhaps individual facts Home glassware 4.18 26041.20 6.41650449969907389 +AAAAAAAAGPPBAAAA More black members would run more central poor phases. Personal responsibiliti Home glassware 8.30 423.06 0.10424121751849724 +AAAAAAAAHBKBAAAA Safe, distinct millions must not deliver at the men. Indeed old claims might put exercises; particular, wooden households should learn clear, lucky votes. Mean, level terms might write bot Home glassware 9.86 7952.69 1.95952840766599957 +AAAAAAAAHOBDAAAA Significant difficulties could observe numbers. Very alone centuries affect forwards by a matters. Glad fields ought to spread hardly british str Home glassware 3.06 501.96 0.12368203457094708 +AAAAAAAAIACDAAAA Novel, small attitudes may warn now however good terms. Aware earnings must eat much; lat Home glassware 2.84 5534.76 1.36375483636523840 +AAAAAAAAIBHDAAAA Cold, old days stem thereby difficult, nuclear men; likely contents shall threaten often outer years. All real or Home glassware 9.08 11902.21 2.93268298009935465 +AAAAAAAAIELDAAAA Now strong fields may not feel. Again Home glassware 3.96 9805.52 2.41606236279008890 +AAAAAAAAIGPDAAAA Even sexual men can clear thereby always male members. Shoulders extract. Negotiations used to alter else Home glassware 3.47 1371.15 0.33784887581073012 +AAAAAAAAIJJDAAAA Conditions could not estimate following problems. Theories get sure; extremely complete scholars ought to thrive only strong, european businesses. Important, social p Home glassware 1.56 6751.07 1.66345141670827100 +AAAAAAAAIMBEAAAA Holes buy then markets. Practical themes ought to escape above australian children. Home glassware 1.43 3401.20 0.83804951785541719 +AAAAAAAAINDCAAAA Willing, due values will chat hardly gmt central records. Necessary, adult stairs make fast in terms of a years. Views would not dig Home glassware 0.24 2373.76 0.58489016332602467 +AAAAAAAAINPCAAAA Moments used to contract really boats. A Home glassware 68.94 1997.56 0.49219516490864023 +AAAAAAAAJIIDAAAA Insects indicate germans. Other, particular properties might Home glassware 4.52 2374.24 0.58500843445638178 +AAAAAAAAJKCEAAAA Persons might live here doctors. Chil Home glassware 2.86 15578.10 3.83841561628351009 +AAAAAAAAJNOBAAAA Materials make apart colonies. Rates make naturally poor, appropriate companies; c Home glassware 0.80 1956.16 0.48199427991533955 +AAAAAAAAJPDBAAAA Used groups ought to fail high from the districts. Immediate, main walls could exploit rights. Therefore late friends ought to try away. In short widespread lakes sh Home glassware 80.17 9287.91 2.28852419657312357 +AAAAAAAAKIDBAAAA Too only affairs put nonetheless big numbers. Rapid students appeal for the Home glassware 9.29 13621.22 3.35624392967263487 +AAAAAAAAKKHCAAAA Good windows say widely actions. Simple, imaginative findings see to a recommendations. Environmental, l Home glassware 4.66 12892.65 3.17672560166371999 +AAAAAAAAKNMDAAAA Japanese emotions speak disabled, new techniques. Experts should not tell only refugees. Years cannot afford well head quarters. Offices make conscious, primary stories Home glassware 7.31 4129.01 1.01738058324126665 +AAAAAAAAKPJBAAAA Full goods should find then. Only able years exploit completely mode Home glassware 2.13 3040.36 0.74913919560946025 +AAAAAAAAMDMBAAAA Sexual, due tensions take quite lucky circumstances. For ever formal districts respond ways. Poor relations should not come correctly in an facilities; important times could look away common Home glassware 42.90 1247.40 0.30735710001553787 +AAAAAAAAMDNAAAAA Bad boys might claim shortly italian, good lines. Times learn additional, sick cards; measures work sometimes pleasant male Home glassware 2.10 3225.77 0.79482388369177617 +AAAAAAAAMHFBAAAA Children want on a paintings. Over nice teachers must not sell. Richly accurate pp. hate as african, fiscal days. Claims eat part Home glassware 7.95 6793.78 1.67397508332817129 +AAAAAAAAMLGAAAAA Always sad weeks would not put close to a masses. Fresh, atomic sides will not help together previous Home glassware 0.83 6893.14 1.69845720731209292 +AAAAAAAAMMPBAAAA As other Home glassware 4.88 2352.12 0.57955810653242499 +AAAAAAAAMNECAAAA Serious branches use. Rich, english bombs keep much vulnerable consequences. Little, furious sales can keep to a gentlemen. As gold customers overlap betwee Home glassware 2.54 3062.18 0.75451560407694385 +AAAAAAAAMNIBAAAA Really different shares ought to help clearly p Home glassware 2.82 6640.72 1.63626137663554805 +AAAAAAAANGAAAAAA There possible newspapers experiment. Annual accounts might visit possible, prime groups; competitive, universal pr Home glassware 1.12 63.36 0.01561178920713843 +AAAAAAAANPPAAAAA Recent, labour complaints must read in a units. Softly old courts rely even. Actual Home glassware 8.70 2861.55 0.70508073556955459 +AAAAAAAAOAIBAAAA Well new carers shall give together with a samples. Individual, central birds find there weapons. Kind details proceed ultimate miles. Unlike, independent months mus Home glassware 0.46 6486.44 1.59824706415326716 +AAAAAAAAOMGDAAAA Overseas businesses conceal gmt in a farmers. Level functions could support all right dreadful processes. Walls buy furth Home glassware 3.81 10274.91 2.53171920836992962 +AAAAAAAAONFAAAAA Mental techniques might prohibit by a chiefs; other, waiting defendants vary else. Now old skills would see. Common jobs will no Home glassware 89.76 2200.15 0.54211297386498769 +AAAAAAAAOOFCAAAA Dogs will cover never. Bitter children restore cheaply upper, short views; other teams shall exist too high customs. Yards must not help now present, coming mines. However federal method Home glassware 3.22 2352.77 0.57971826535478358 +AAAAAAAAPBFBAAAA More than divine areas will control together from Home glassware 4.90 563.56 0.13886016296677611 +AAAAAAAAPBMDAAAA Surely national arguments address working, soviet effects. Again central parents say english rules; carefully military chang Home glassware 8.61 13637.98 3.36037356330760394 +AAAAAAAAPGHAAAAA Classical, attractive employers want only prices. Financial approaches used to hear considerable votes. Bo Home glassware 2.50 13555.23 3.33998411323041478 +AAAAAAAAPKCBAAAA Other patients see normal colleagues Home glassware 4.62 1970.54 0.48553748586228795 +AAAAAAAAPKHCAAAA Newspapers ought to pursue. Well rare criticisms used to tell so. Powerful, new matters touch. Home magic brothers can read now rather supreme rats. As evolu Home glassware 4.99 1537.58 0.37885692628017534 +AAAAAAAAAANAAAAA Surely additional years work never remote, great bits; women deal in a judges. Far ethnic hands might help afterwards already dead awards. Rich, social experts target social children. National Home kids 0.50 361.08 0.11815869948988022 +AAAAAAAAABBAAAAA Yet black costs must not judge here lively variables. Full, po Home kids 1.68 3938.44 1.28880289248621866 +AAAAAAAAABNCAAAA Proud investors may not visit extremely. Alone, everyday houses move widely global countries. Only single gardens come further shadows. Scottish, wo Home kids 2.68 31.68 0.01036686496022877 +AAAAAAAAAIEEAAAA Total, new savings would make short, popular consultants. Short, other contracts might discuss for a Home kids 9.91 1600.56 0.52376229105883094 +AAAAAAAAAKADAAAA Effective, free arrangements will build social, possible agreemen Home kids 4.30 2319.90 0.75915688198341950 +AAAAAAAAAKGDAAAA Enterprises shall not influence perhaps delighted, big police. Novels keep early temporary bacteria; rates will not cope men Home kids 3.57 6583.08 2.15422668504996302 +AAAAAAAABAAAAAAA Agricultural sites will not provide skills. Again Home kids 0.55 5015.40 1.64122394323015739 +AAAAAAAABDKCAAAA Conservatives tell effectively in a parties. Dir Home kids 6.35 8063.47 2.63866491795631001 +AAAAAAAABFABAAAA Too old Home kids 0.95 114.66 0.03752098283900982 +AAAAAAAABLDBAAAA Following occasions see then only real lovers Home kids 5.63 6310.36 2.06498263795546836 +AAAAAAAACCPDAAAA Permanent details would help also off a owners. External children used to listen like a years Home kids 30.73 6001.32 1.96385334668939829 +AAAAAAAACFKCAAAA Farmers might not assume now to the tanks. For Home kids 3.80 11826.88 3.87019153601106270 +AAAAAAAACGOAAAAA Local farmers skip also shoulders; things ought to seem so only applications. Foreign, voluntary voices may not find new Home kids 3.96 2251.62 0.73681314651989612 +AAAAAAAACHLCAAAA Now close items become already against a groups. Authorities would work as well natural, dependent parties. Operators should not fall l Home kids 5.59 7257.25 2.37483998524685165 +AAAAAAAACIAEAAAA Appropriate items take mediterranean centuries. High, very days see ways. Careful, technical minds remai Home kids 4.98 10259.21 3.35719206656024705 +AAAAAAAACJGAAAAA Dire Home kids 4.41 1733.90 0.56739605917110697 +AAAAAAAACKPBAAAA Short areas would not improve below to the measurements. Vo Home kids 0.36 18342.34 6.00229046195084044 +AAAAAAAACMDCAAAA As beautiful children strike really natural services. Too assistant pow Home kids 3.30 2799.11 0.91597207635182954 +AAAAAAAACPKBAAAA Even growing seats may build for a times. Obvious, different systems require american settlements. Evil yards support worldwide possible members. Courses could build also users. Alm Home kids 4.28 2619.47 0.85718723981598684 +AAAAAAAADDCCAAAA Gold, young schools shall not go far human hands. Aware terms brush almost. Real years treat early. Edges cannot stop still british assessments. Very royal skills shall say already other Home kids 5.63 4448.98 1.45587041890020849 +AAAAAAAADGDBAAAA Dogs hang perhaps chief, visual brothers. Minimum, small families shall work strong mountains. Small, defensive factors make by Home kids 5.44 2978.61 0.97471109972181264 +AAAAAAAADJHAAAAA So dependent things distinguish again new subjects. Critical, firm centuries increase then institutions. Effects allo Home kids 1.59 10537.48 3.44825227844417572 +AAAAAAAADMDBAAAA Turkish, old women must improve far from full, new changes. Days keep again exactly secondary visitors. Things used to make great, other notes. General, hig Home kids 1.38 355.77 0.11642107155620551 +AAAAAAAAECJCAAAA Examinations reduce other, late things. Police should help very strong boxes. Annual, sole reports might benefit fortunate, total seats. Never rural shapes shall cease pictures. Physical periods wi Home kids 3.60 1189.98 0.38940536506859327 +AAAAAAAAEHDAAAAA Likely products ought to work other, considerable arrangements. Also other funds kill possible, royal patterns. Old, good files know palestinian colours. Northern Home kids 1.60 3252.96 1.06448854296167261 +AAAAAAAAEKMDAAAA Minds could not decide later avail Home kids 2.36 7178.10 2.34893918469122957 +AAAAAAAAEKNCAAAA Teams make never features. Now russian individuals may reproduce indeed other visual lakes. International legs drive also married views. Catholic populat Home kids 8.74 5328.40 1.74364909261625606 +AAAAAAAAEMNAAAAA Healthy, delighted conclusions may offer experienced condi Home kids 4.30 1952.10 0.63879915053227863 +AAAAAAAAFENAAAAA Reasonable pictures could not try features. Unexpected politicians remember always. Serious buildings pay thereafter aged a offers. Large, material products go tomorrow interesting, individual re Home kids 44.54 107.20 0.03507979557249130 +AAAAAAAAFJEBAAAA Equal supplies could get easily still new years. Equivalent, national policemen shall appeal. Tables would Home kids 7.14 13784.20 4.51069886315610630 +AAAAAAAAGDBDAAAA Hours get skills; foreign, positive events disguise currently apparent prices; other programmes may sink honours. For instance var Home kids 7.04 2430.74 0.79542781986826031 +AAAAAAAAGNIBAAAA Apparently effective deals could stand Home kids 0.92 1924.93 0.62990812398652687 +AAAAAAAAHBJCAAAA Funny times go actually much old details. Military parameters tell so studies. Values u Home kids 4.41 1907.42 0.62417820588508729 +AAAAAAAAHDIBAAAA Levels contact in a sides. Companies must not count with an boxes; yet physical days happen never from a opera Home kids 8.77 13024.65 4.26214607652284354 +AAAAAAAAICLCAAAA Questions seem strongly. Political years establish guilty centres. Necessary, pale eyes used to generate social, particular assets. Conditions help as firm directors. Persona Home kids 9.37 8639.50 2.82716318888562125 +AAAAAAAAIEGEAAAA Subsequent qualities say broadly good objectives. Odd workers ought to make commonly therefore intact times. Objectives will not hold just with the types. B Home kids 0.64 3035.53 0.99333742401272873 +AAAAAAAAIKMBAAAA Soon artificial notions think no longer lights; clearly late members could not trace good countries. Cultures can proceed away wealthy Home kids 2.38 3035.43 0.99330470032282902 +AAAAAAAAJHMDAAAA Appropriate, new ad Home kids 3.99 396.54 0.12976251992831810 +AAAAAAAAJNADAAAA Ruthlessly empty times shall not focus to a lectures. Skills involve even; boring periods re Home kids 0.63 1007.86 0.32980898102323771 +AAAAAAAAKBACAAAA Lists could play round, new roads. Soon national calculations think usually at first similar benefits. Skilfully specific practitioners will believe that is bars. More immediate Home kids 8.24 3098.01 1.01378318546206881 +AAAAAAAAKINDAAAA Suggestions must see much large assessments. Disabled charges might claim besides wide, white passengers. Democratic, wide relationships test little years. Working, bri Home kids 0.50 934.46 0.30578979263684908 +AAAAAAAAKMDDAAAA Strong settlements should close here. Forms may seem quickly other unions. Places employ difficult banks. Women must not accept too areas. Vast possibilities know; never healthy subjects cancel most j Home kids 1.95 10592.00 3.46609323417749873 +AAAAAAAAKNJDAAAA English requests serve also intervals. More late cards might make only other companies. Tragic lights learn more royal, attractive studies. Businessmen ought to defend close po Home kids 1.59 17495.72 5.72524515852189842 +AAAAAAAALDODAAAA Goals help still human plates. Practical groups t Home kids 4.79 16455.90 5.38497768620671273 +AAAAAAAALIFEAAAA Full, good fans might not pose of course parts. Daily Home kids 85.83 7041.80 2.30433679535792207 +AAAAAAAALJACAAAA Due years show just ashamed homes. Large, australian parties suit there automatic grounds. Sexual steps might not mean today technical molecules. Al Home kids 6.52 4853.82 1.58834900509020269 +AAAAAAAALLDAAAAA Then dark tactics should not follow then. Ashamed, g Home kids 1.43 11882.09 3.88825828520469372 +AAAAAAAAMCNBAAAA Vocational, political styles run incorrectly indeed only hands. Complete, confident employers expect big owners. Inc times should stop more; consi Home kids 8.09 3606.10 1.18004898147351569 +AAAAAAAAMKJAAAAA Formal matters must admire much. Capable rules rise however. Harder only studies would show more. Old stones oppose common, secure police. Opinions come grey, appropriate systems. Eye Home kids 6.02 261.24 0.08548736749400772 +AAAAAAAANBJCAAAA Soft, good estates must not join most likely, accused pieces. Coming, historical pictures arrange; best old loans cannot Home kids 6.24 6536.61 2.13901998635356684 +AAAAAAAANGIBAAAA About american members provide certainly increased, special experienc Home kids 0.99 5029.15 1.64572345059136780 +AAAAAAAAODPDAAAA Trying, ti Home kids 3.34 16043.89 5.25015281145090918 +AAAAAAAAOGJCAAAA New, other companies could take always political years. Important charges wait sure other aspects. Legal grounds may not worry to Home kids 6.49 5131.46 1.67920305772776318 +AAAAAAAAOPCBAAAA Windows recommend never internal cells. Mutual, other moments should not see levels. Necessary, national costs shall not walk light, high types; more digital days might continue. Home kids 2.75 8373.49 2.74011490138339726 +AAAAAAAAPADEAAAA Fresh, f Home kids 1.45 4190.94 1.37143020948299155 +AAAAAAAAPICDAAAA Quickly wrong facilities prepare as. Similar surveys look hopelessl Home kids 3.16 116.22 0.03803147240144533 +AAAAAAAAAHECAAAA Remote, left figures used to feed on a records. Over economic depths must understand in particular at the ranks; degrees can think go Home lighting 2.60 5654.38 2.08346575200781715 +AAAAAAAABMMDAAAA Lovely letters would require now questions; communities will add years. Emotional, traditional times make for a patterns. Perhap Home lighting 8.69 2656.29 0.97876146321981272 +AAAAAAAACBPBAAAA Moving, powerful drugs use so blind honours. Efficient, other seconds look just rare, planned homes. German, specified sons reside further red weeks. Available lists undergo young, milit Home lighting 0.67 10412.96 3.83685665573012774 +AAAAAAAACGIBAAAA Different men may not inform by now between a eyes. Members can cause new minds. Strong, chief rooms will carry high lessons; natural molecules accept here because of a talks. Eyes may disc Home lighting 0.56 7704.59 2.83890530849746709 +AAAAAAAACHHAAAAA Incidentally immediate flames earn. Friends influence certain, potential men. Early, opening conventions should see per a agencies. Economic, senior practitioner Home lighting 1.62 616.89 0.22730506045863602 +AAAAAAAACJGDAAAA Original others get industrial yar Home lighting 1.48 6297.95 2.32060157486013180 +AAAAAAAACJJDAAAA So soviet years get. Good things must appreciate well real churches. Overseas, constant boxes complete for no months. Subjects may not suffer widel Home lighting 5.50 178.36 0.06572019417303299 +AAAAAAAACMCDAAAA Important, toxic commun Home lighting 0.33 431.67 0.15905716650971714 +AAAAAAAADFFDAAAA Prisoners give fundamental months. Opportunities grasp capital actions. British iss Home lighting 5.72 5860.48 2.15940728609091930 +AAAAAAAADKDDAAAA Under way long interpretations might take yesterday. Little little shares get quickly families. Measures occur. Forward daily hands m Home lighting 2.56 2458.11 0.90573820642898698 +AAAAAAAADNMDAAAA New, future communities should make yesterday particular, primary relations. Significant students mea Home lighting 83.07 7959.15 2.93270286752800800 +AAAAAAAAEBDAAAAA Opportunities drop cars. Officials change as for a inches. Other, american societies take straight leading, total posts. Agreements get Home lighting 65.24 13670.55 5.03717874216279499 +AAAAAAAAEBFBAAAA Vital problems may lig Home lighting 60.33 6799.66 2.50546633500003077 +AAAAAAAAECIAAAAA Rather american gentlemen might generate rather in a studies. Enough current negotiations used to co-operate nearly rough main rivals. Dramatic, overall weeks used to provide too other, great meal Home lighting 7.69 3528.80 1.30025466022538018 +AAAAAAAAEDCEAAAA Also new colonies go unhappily eggs; typically modern centres would provide then Home lighting 0.51 5329.54 1.96377216670187391 +AAAAAAAAEJNCAAAA Prayers increase ever depths. International, official member Home lighting 7.88 4324.07 1.59328728424415089 +AAAAAAAAEMPDAAAA Sick, old-fashioned birds might think there imports. Grant Home lighting 7.01 5314.03 1.95805720700449927 +AAAAAAAAFDFDAAAA Common contracts will undergo for the goods. Generous, long laws shall not reach less traditional men. All pla Home lighting 3.29 973.56 0.35872702533694772 +AAAAAAAAFDJCAAAA Front shelves produce more at a principles; previously everyday birds avoid on a matters. Up Home lighting 18.01 4993.08 1.83979696748983826 +AAAAAAAAFGIAAAAA Problems should prevent finally in a effects. Now economic men sign. Royal, permanent teeth can start colonies. Geographical eyes wi Home lighting 9.41 5689.57 2.09643218861327258 +AAAAAAAAFJJCAAAA Essentially everyday lines sing s Home lighting 6.37 2165.33 0.79785774864708186 +AAAAAAAAFLECAAAA Famous, attractive arms shall go publicly just democratic men. Importantly private ministers ought to write. Levels bring most true, adjacent days. Successful, particular constraints may pl Home lighting 3.16 2680.48 0.98767473691932868 +AAAAAAAAFNBAAAAA Just familiar police work virtually rare fruits; blind police might not succeed possible, stable churches. Senior communications light old, economic activities; almost direct characters ca Home lighting 2.42 14392.73 5.30327994101837339 +AAAAAAAAGAOAAAAA New kinds will go wholly great, occasional models; efforts may seem then too local homes. However religious co Home lighting 4.11 408.39 0.15047919992332890 +AAAAAAAAGBNCAAAA More possible newspapers Home lighting 9.78 3183.02 1.17284532662394854 +AAAAAAAAGEJBAAAA Of course high Home lighting 4.02 405.11 0.14927062043864877 +AAAAAAAAGFGAAAAA Further high men can give stairs. Controversial, great fingers hate sometimes generally ancient books. Other dogs woul Home lighting 6.69 1549.44 0.57092115754353125 +AAAAAAAAGMLCAAAA Visual, sensible rates know instead excellent honours. Other, inc christians fill plans. Girls may not make to a institutions. Days could build appropriate, small statements. Left, runnin Home lighting 1.12 8531.28 3.14351523965302125 +AAAAAAAAHBCAAAAA Propos Home lighting 1.14 5525.76 2.03607322355673225 +AAAAAAAAHLICAAAA Significantly severe hundreds let right. Domestic, good approaches like of course later main records. General firms will preve Home lighting 17.01 2134.46 0.78648309965559538 +AAAAAAAAHOHAAAAA More great societies press. Years make still other, lively standards. Decisions may strike to Home lighting 0.43 2644.48 0.97440984013625407 +AAAAAAAAICJBAAAA Unusual, fierce imports can press fine rural contents. Perhaps public Home lighting 4.21 7474.73 2.75420894253753570 +AAAAAAAAIFGBAAAA Middle-class years record also recent problems; certain, mild others can show. Matters will influence solely books. Loca Home lighting 6.43 2611.80 0.96236826161206301 +AAAAAAAAILCBAAAA Able, double cells monitor quickly tomorrow direct men. Different weeks used to become n Home lighting 7.19 187.35 0.06903273367525079 +AAAAAAAAJABAAAAA Legal conventions ought to work in accordance with a cases. Together left books may not come sure subsequent things. Short, real products deal excessive, im Home lighting 5.79 9924.55 3.65689253801286467 +AAAAAAAAJBDDAAAA International, final writers must learn political eyes. Immediate times reach also also wrong requests. Isolated years may not plan yesterday minutes. Long, old researc Home lighting 0.62 4542.45 1.67375362200770182 +AAAAAAAAKEPAAAAA Alone new conditions will recognise personal, hot others. Sooner scottish eyes permit probably only advanced cases. Never impossible services use again direct Home lighting 4.82 8731.18 3.21717226373459388 +AAAAAAAAKFFCAAAA Usually severe kinds cost incidentally conclusions; normally continuing concentrations ought to tell amer Home lighting 0.90 8588.69 3.16466906532847440 +AAAAAAAALENDAAAA Empty, splendid pounds make relatively skills; public, simple exchanges might exploit simply. Basically quiet perceptions shall not sleep. Old, alone individuals get permanent, new minerals. Fo Home lighting 2.10 4427.11 1.63125436659215111 +AAAAAAAAMALAAAAA White, fair artists take. Simply silent years could create general, alternative issues. Deliberately natural moves take so n Home lighting 5.13 1353.00 0.49853903743055412 +AAAAAAAAMBDBAAAA Regular villages will raise finally small, rich terms; working-class, smooth states may violate european interests; discussions should not tell particularly times. Delightful, previous obje Home lighting 2.57 1509.56 0.55622659966272526 +AAAAAAAAMGMBAAAA Happy sorts should care. Definite, sensitive pages should happen else smooth clouds. Local, legal years might not represent easy unfair clothes. Poor, other powers change only fo Home lighting 8.25 6600.48 2.43207460885411963 +AAAAAAAAMMIAAAAA Plates shall think; new, economic pupils collect entirely. Really powerful books develop yet girls. Best unlik Home lighting 3.44 2151.42 0.79273233991784386 +AAAAAAAAMNDEAAAA Writers say. Spanish, local targets find able weapons. Figures would win most into the effects; as steady workers shall understand. Social point Home lighting 5.26 5754.60 2.12039375077447653 +AAAAAAAAMNNCAAAA Fiscal, occasional subjects ought to provide ill altogether royal stocks. Individual students save within a students. Home lighting 2.33 6565.32 2.41911922632931676 +AAAAAAAANDPAAAAA Villages Home lighting 3.15 5303.78 1.95428039611487386 +AAAAAAAANEJCAAAA Rich, logical Home lighting 7.93 2820.76 1.03936361805070942 +AAAAAAAANMKCAAAA Residents a Home lighting 4.83 13929.25 5.13250176432338949 +AAAAAAAANPJDAAAA Ridicu Home lighting 4.71 6980.98 2.57227719846411656 +AAAAAAAAODEEAAAA Behind aware variables cannot bring into a contents. Different, electronic mistakes measure; however additional students should like. Interesting sales wo Home lighting 1.37 1624.72 0.59865953059436060 +AAAAAAAAOFJAAAAA Common feet cannot send at a engines. Orders should think prime, conservative cell Home lighting 2.52 2080.16 0.76647521367445784 +AAAAAAAAOJJDAAAA Emotional areas make then new targets. Difficulties should halt much. Military circumstances might mount very much white systems. Other holidays drag further through a Home lighting 6.63 10785.78 3.97422940069306875 +AAAAAAAAPAEDAAAA Yet young minutes could not walk here; enough pale others may not Home lighting 1.89 7242.84 2.66876458378678093 +AAAAAAAAPGDBAAAA Difficulties apply just initially high surroundings. Enough usual requirements assist repeatedly for a students. Directions make too through the flowers. More national historia Home lighting 9.68 372.50 0.13725483476931368 +AAAAAAAAPHCEAAAA Always top problems change almost expensive women. Supreme, industrial discussions Home lighting 4.16 1004.00 0.36994323250574748 +AAAAAAAAPIBBAAAA Discussions emerge so annual lessons. Good, early faces play really legislative products. Cold, private societies understand clearly ahead fat manufacturers. Abstract causes become so as executi Home lighting 9.11 4351.81 1.60350862415422005 +AAAAAAAAABOCAAAA Approximately senior colours cannot encomp Home mattresses 4.73 2262.11 0.80877478687478841 +AAAAAAAAAKBDAAAA Facilities shall look just much quiet clients. Specific prices should explain on a ways. Aspects ought to establish ill high chains. Suitable, enormous areas c Home mattresses 0.21 4913.00 1.75655053375646430 +AAAAAAAAAMNBAAAA Sufficient, united companies earn either for no months. Comfortable, big tears come spiritual, old bir Home mattresses 6.95 6514.82 2.32925107843014222 +AAAAAAAABLHAAAAA Complex, social miles cannot tie faces; probably future universities get objectives. Given settlements cannot g Home mattresses 4.30 100.50 0.03593188044830545 +AAAAAAAACHEEAAAA Even widespread figures help also new, coloured trees. American, potential chapters may get actually years. Genes alter sides. Fingers develop par Home mattresses 4.87 NULL NULL +AAAAAAAADGBAAAAA Dark companies stem in a offices. However multiple hours will preserve most long copies. Over mil Home mattresses 4.19 265.00 0.09474575441592979 +AAAAAAAADHAAAAAA Early children shall not burst environmental Home mattresses 29.32 1972.12 0.70509432905186207 +AAAAAAAAEAIBAAAA Strong t Home mattresses 3.26 972.30 0.34762753591927748 +AAAAAAAAEHGAAAAA Also unknown books want very structural eyes. Well existing years could not buy much constant simple clients. Clouds find; ordinary, magic years prevent surely. Pensioners Home mattresses 0.47 5228.57 1.86937663836414340 +AAAAAAAAEHLBAAAA Central, new children agree strangely legitimate, full values. Underneath adequate rights avoid just rough museums; dead, local shareholders spare various forces. Small letters force finally women. Home mattresses 2.58 4991.57 1.78464175611291563 +AAAAAAAAEJBCAAAA Terms connect too all personal doctors. Current, new hours used to revive for the schools; practical, willing leaders discuss cases. Ago new structures must answer. More willing minutes claim more. F Home mattresses 5.91 5652.60 2.02098057136409324 +AAAAAAAAEPKDAAAA Physically useless findings continue other critics; perhaps young forms substitute coins; arms command Home mattresses 0.77 13274.08 4.74589707085813303 +AAAAAAAAFBOCAAAA Groups make in t Home mattresses 4.98 5572.29 1.99226724480883542 +AAAAAAAAGEGBAAAA Skills should spend twins. Certain, industrial homes will get to a rights. Decisions could buy politically so difficult differences. Running magistrates cannot respect thickl Home mattresses 7.20 4964.20 1.77485612857191941 +AAAAAAAAGGFDAAAA Here extra efforts ensure eyes; merely little periods will not loosen home past a boys. Just local aspects must reclaim. Standard qualities might not roll today. Military, national clothes must go wid Home mattresses 3.34 4129.43 1.47639985153876580 +AAAAAAAAGHKAAAAA Possible, rich d Home mattresses 4.63 10156.22 3.63116500344963929 +AAAAAAAAGIBEAAAA Japanese years escape so good objects. Tiny features see then proud heads; abroad full secrets might not re Home mattresses 0.95 2753.98 0.98463363300521627 +AAAAAAAAGOMDAAAA Past, interior years fetch accidents. Away internal feet would not organise so square collective rocks. M Home mattresses 6.31 3321.81 1.18765054519388575 +AAAAAAAAGPDCAAAA National, difficult pain Home mattresses 0.37 987.66 0.35311921436391401 +AAAAAAAAHCGBAAAA British differences discuss almost in the advantages; in particular international operations go then in a architects. Regional, fair costs commit merely political items. Then difficult travel Home mattresses 3.06 430.92 0.15406732261476401 +AAAAAAAAHNGCAAAA Never arab policies follow only. Valuable employees might shed. Recently relative costs order just with a areas; sessions may come somewh Home mattresses 6.84 7661.12 2.73908903423006793 +AAAAAAAAIEFAAAAA Perhaps blank models work certain Home mattresses 4.17 1990.47 0.71165502563122929 +AAAAAAAAIEJBAAAA Keys must not read political, italian farmers. Red, single years should play however at the dates. Authors disturb no longer for a purposes. Ever essential agencies will answer as fundame Home mattresses 42.14 5401.80 1.93131175926026233 +AAAAAAAAIHLDAAAA Payments forget. Doubts make respects. Considerable, available states should go here. Only public pages might differ. In Home mattresses 3.45 2289.13 0.81843527851372585 +AAAAAAAAIJOBAAAA Well able areas examine respectively previous services. Surprised computers ought to love british, sole appeals. Common, similar inhabitants finish from a seco Home mattresses 7.94 3465.86 1.23915290716979022 +AAAAAAAAIMDAAAAA Social councils used to determine yet at the boats. Persons ask alive months. Individual, considerable rooms note cases. Then only policies may look to a Home mattresses 4.91 4363.94 1.56024448122963257 +AAAAAAAAJCPCAAAA Films must ta Home mattresses 6.04 6064.00 2.16806888595546499 +AAAAAAAAKJGDAAAA Educational hopes appear more high others; black thoughts might close always in a officials; close years base top workers. Regulations ask over high widespread Home mattresses 3.52 15000.77 5.36324253007489455 +AAAAAAAAKNFDAAAA Vital arms generate slow, neat judges. Specially simi Home mattresses 4.42 10296.27 3.68123724083058633 +AAAAAAAAKNIBAAAA Closely blind winners might come similar, local crops. Very difficult evenings can stretch only ago naked hands. Sufficient, similar Home mattresses 6.05 13831.69 4.94526001470668627 +AAAAAAAAKOCCAAAA Natural beans look often bacteria. Square, small items must negotiate for the forces. Hence chief ha Home mattresses 6.40 161.10 0.05759826806191052 +AAAAAAAAKPKCAAAA Large, very materials like otherwise long, rough concepts. Sources give as local children. Rapid customers remove gently downwards short expressions. Behind national crimes confess n Home mattresses 7.74 1076.05 0.38472139260098583 +AAAAAAAALEOCAAAA Growing, social objectiv Home mattresses 7.70 8.96 0.00320347909270464 +AAAAAAAALFIBAAAA Ago new studies shall not apply of course small forces. Dead parts used to point on a students. Then other students should pay only Home mattresses 8.92 16657.18 5.95546070015825401 +AAAAAAAALGIDAAAA Good, ethical interests stress now because of the eyes; patients used to give so hills. Social operations will pronounce basic ideas. British friends store too p Home mattresses 0.68 2433.04 0.86988758612880682 +AAAAAAAALKOCAAAA Following, combined cells must ease of course continued changes. German te Home mattresses 5.91 785.92 0.28099088041723599 +AAAAAAAALMHBAAAA Old words cannot force. Equal, capital problems would not produce; great, competitive things congratulate only times. Vice versa unemployed complaints will say previous gardens. Difficult, uncomfort Home mattresses 1.57 1412.84 0.50513430818491411 +AAAAAAAAMDIDAAAA Now comfortable grounds bowl much only double groups. Good talks must not support somewhat; used, linear Home mattresses 5.00 5416.79 1.93667115117986530 +AAAAAAAAMMCCAAAA Respectively excellent things speak reliable, historical movements. Masters respond. Cheap ideas should featu Home mattresses 3.37 5563.35 1.98907091633910557 +AAAAAAAAMMEEAAAA Prisoners ought to leave. Main items should not strengthen ago allowances. Ideas provide together between a patients. Regional, english conditions argue also in a minutes; ordinary trials become lon Home mattresses 36.96 6326.30 2.26184930626979851 +AAAAAAAAMPNBAAAA Cases move so very natural tories. Therefore political cells win there already eastern events. Extra questions encourage skilled efforts. Serious, physical clothes would Home mattresses 80.68 751.60 0.26872041139250123 +AAAAAAAANAFBAAAA Individuals recognise. Really elegant relations should extend totally types; attitudes would relate muc Home mattresses 7.09 1139.56 0.40742819585742244 +AAAAAAAANCMBAAAA Evidently super tales may risk just; others match maybe. Lovers describe anywhere Home mattresses 2.32 9619.86 3.43939959651179740 +AAAAAAAANLJBAAAA Minimum words Home mattresses 6.86 4696.59 1.67917721785777990 +AAAAAAAANOFCAAAA Other, extra notes alter so social ways. Different, preliminary parts take so diffic Home mattresses 3.40 10150.42 3.62909132278695101 +AAAAAAAAOAFAAAAA So social decisions fulfil again comparative times. Academic governments ought to arise then on a decades. Home mattresses 1.81 1346.52 0.48142284240051991 +AAAAAAAAOICBAAAA Often presidential councillors used to take in the friends. Exact, rich visits used to want sophi Home mattresses 0.41 8719.30 3.11742134520308145 +AAAAAAAAOIIBAAAA Constant, domestic things might worry like a minutes. Literary, kind sales tell however emotional branches. Too specific troops may not bring most fair unknown owners. Issues look official Home mattresses 0.51 148.32 0.05302901998102153 +AAAAAAAAOJJCAAAA Even successful questions continue indian areas; good, good jobs get nice, famous interests. Labour, generous circumstances help good changes. Strict, vulnera Home mattresses 2.55 2079.26 0.74340021632779686 +AAAAAAAAOLCEAAAA Good, full governments consider elsewhere genuinely Home mattresses 0.33 11909.49 4.25801364059989293 +AAAAAAAAOMLDAAAA Days should conti Home mattresses 3.57 1697.42 0.60688052249316052 +AAAAAAAAOODAAAAA Personal, national arts ought to rely still strategic, dead instruments. Finally federal spots remember. Laws Home mattresses 3.72 13796.99 4.93285368384543056 +AAAAAAAAOOJBAAAA New products should not see. Much separate subjects give at least existing implications. Similar corporations might turn years; local Home mattresses 3.84 1888.50 0.67519757439427698 +AAAAAAAAPBMAAAAA Other parties will add regional, special owners. Little administrative horses may indicate; Home mattresses 1.41 23082.32 8.25264838516945064 +AAAAAAAAACMAAAAA Often local men ought to suppress trousers. Angry studies might cool seeming Home paint 0.70 4572.36 1.91646328201692969 +AAAAAAAAAGODAAAA Worthy, rich types force both shy years. Tropical, personal views might work. Other eyes ought to administer neve Home paint 0.28 12758.19 5.34747978724238078 +AAAAAAAAAHAEAAAA Rural others come as through a estimates. Publications should worry really powerful Home paint 3.24 4960.42 2.07911511634744823 +AAAAAAAAAHEAAAAA Early, dangerous weeks live still to a changes. Vari Home paint 2.74 12614.97 5.28745042138963413 +AAAAAAAABECDAAAA Perhaps de Home paint 1.44 1475.69 0.61852209813740890 +AAAAAAAABMKCAAAA Clinical, national residents might cry; objects ought to justify only relatives Home paint 7.77 2688.57 1.12688976505180184 +AAAAAAAABOOCAAAA Equal forces tell together feet. Never new police cannot place hardly big, independent years. Then old choices ought to afford especially; parties accept Home paint 6.51 6336.50 2.65588658515520978 +AAAAAAAACBPCAAAA Always huge concessions express directly ameri Home paint 4.52 9357.30 3.92202754569128769 +AAAAAAAACCEDAAAA Average decisions shall see. Lovely, wide temperatures prepare in a regulations. Right arms ought to make now applic Home paint 57.27 3310.24 1.38745711507049343 +AAAAAAAACJBCAAAA Vast, separate feet wear financially other, dangerous workers. Other, old genes spin for instance ordinary purposes. Events could focus anywhere under fresh countries Home paint 7.37 10616.13 4.44965473893533925 +AAAAAAAACNPBAAAA Quickly far walls shall see gold, true patients. Above bad pensions will insist as round detailed degrees. Free, Home paint 0.70 809.31 0.33921495655834654 +AAAAAAAADCHBAAAA Probably political hands may park easily. Little, suitable officials apply today men; women ought to take to the provi Home paint 6.55 2700.80 1.13201585878437474 +AAAAAAAAEBOBAAAA Special words should tell by a follower Home paint 1.68 592.00 0.24813143824065086 +AAAAAAAAECABAAAA Both usual effects include repeatedly low, possible practices. Professional, past countries retain yesterday ways. Equally old Home paint 0.84 1006.06 0.42168093708849528 +AAAAAAAAECEDAAAA Capital areas judge almost active, numerous f Home paint 9.32 661.29 0.27717371417932434 +AAAAAAAAEEODAAAA Pale, original yards agree social, little generations. Weeks used to include now oral shows; Home paint 2.40 5882.28 2.46550438603752661 +AAAAAAAAFKDBAAAA Appropriate, like v Home paint 4.82 372.76 0.15623897790301523 +AAAAAAAAGABEAAAA Attitudes must build ge Home paint 45.77 9930.33 4.16220788024372040 +AAAAAAAAGBEDAAAA Very difficult parts ought to know else areas. Members could not comment of course male, popular girls. Primary, worried actions might send indirectly elsewhere hard children. New resou Home paint 3.98 770.04 0.32275529172775471 +AAAAAAAAGCJCAAAA Careful universities may find cultural methods; artificial, apparent sections ought to tell highly reforms. Medical, glorious studies shall not agree straight almost actual states. Enough n Home paint 4.20 103.50 0.04338108759781649 +AAAAAAAAGEGDAAAA Players shall mean with a rights. Occasionally popular enemies worry. In general basic patients get perhaps parts. Other varieties enjoy thousands; classes shall not spend as for the families. New f Home paint 2.13 5837.14 2.44658436387167698 +AAAAAAAAGKBDAAAA Students wi Home paint 2.79 4724.08 1.98005534588495595 +AAAAAAAAGKHDAAAA For example clear witnesses used to enjoy yet international, environmental computers. Ill Home paint 9.67 59.46 0.02492212046923835 +AAAAAAAAIABEAAAA Opposite youngsters see altogether. Plans may not say to the problems. Popular, new lands might create cha Home paint 4.08 7043.01 2.95201385277582167 +AAAAAAAAIHFCAAAA Objects sell so yellow ru Home paint 1.47 1136.47 0.47634110746174406 +AAAAAAAAIMCDAAAA Only horses can forget meanwhile animals. Rich exception Home paint 67.74 386.10 0.16183031808228935 +AAAAAAAAIOCBAAAA Responsible, useless sources explore there. Serious, conventional fields could defend once again famous efforts. Officials call as notions. Big, ap Home paint 3.14 8952.05 3.75217067855104485 +AAAAAAAAJCIAAAAA Aware groups could finish services. Companies make also glad, top ways; t Home paint 3.27 1574.90 0.66010507108986663 +AAAAAAAAJEGAAAAA Ever insufficien Home paint 2.77 3898.21 1.63389941531095878 +AAAAAAAAJGABAAAA Se Home paint 7.48 13291.94 5.57119626555479193 +AAAAAAAAJJHDAAAA Windows avoid. Always noble funds should lead nowhere able initiatives. Under new groups wait plans. High enterprises could know inadvertently different, main Home paint 8.31 804.05 0.33701027519830292 +AAAAAAAAKEFEAAAA Human honours tell requests. Effective, late crimes know on a courses. Adequate, typical men should not tend already in a nerves. Home paint 1.35 7526.60 3.15470622138865334 +AAAAAAAAKEJDAAAA Patterns might not maintain. Great, vast eyes say still different views. Easily national plants develop together with the cities. Able g Home paint 21.04 8770.96 3.67626844518787008 +AAAAAAAAKGOBAAAA Possible guests want only; organisations weigh however explicitly c Home paint 4.69 2761.50 1.15745771402290094 +AAAAAAAAKLDBAAAA Letters state on a chains. General, criminal cases shall look unknown months. Special, poor nights give as ever Home paint 7.47 3235.66 1.35619758354348711 +AAAAAAAAKOBBAAAA Also inc goods could not lay Home paint 2.41 2540.30 1.06474373743703612 +AAAAAAAALCBCAAAA Additional companies visit. Grey opportunities may not look numbers. Entire, british models assist also great quarters. Little males show Home paint 51.57 13562.60 5.68464095318015436 +AAAAAAAALFPDAAAA Communist, different demands die perhaps kinds; likely, public forests should make moral, nice faces. Efficient, central services can p Home paint 0.27 668.17 0.28005740386698596 +AAAAAAAALPPCAAAA Effectively initial representatives amount dark areas; comprehensive, christian words will not want hearts. There judicial men explain r Home paint 4.54 5116.69 2.14461427150600652 +AAAAAAAAMDABAAAA Reasons will look probably key students. Now very bones us Home paint 3.58 54.00 0.02263361092059991 +AAAAAAAAMFDCAAAA Features need stages; french cells come hours. Still small beliefs look scarcely electric, good producers. Churches receive for the seats; businesses get appropriate, high ways. Purpo Home paint 2.89 7559.52 3.16850434123135981 +AAAAAAAAMGJBAAAA Managers ought to express so new faces. Universities should not appear at a stories. Accidents dismiss only single times. Other, current companies could not meet effectively that is to say perfe Home paint 0.74 6272.75 2.62916635004061266 +AAAAAAAANGCBAAAA There blue items see in a conditions; lives ask silent countries. Here necessary months may encourage free Home paint 7.02 4828.00 2.02361247267882156 +AAAAAAAAOEABAAAA New dollars might end old relationships. Other, gentle groups Home paint 8.34 2369.97 0.99335146062026237 +AAAAAAAAONBBAAAA International years collect respectively affairs. Exter Home paint 69.84 5908.06 2.47630983954739820 +AAAAAAAAONNBAAAA Colleagues attach th Home paint 9.80 2499.83 1.04778110347487541 +AAAAAAAAOPEEAAAA Furthermore additional Home paint 8.18 1563.59 0.65536458702482987 +AAAAAAAAPDOCAAAA Months find there costly foreigners. White, particular changes used to share away in a subjects. Muscles make fully less gold fingers. Norm Home paint 4.97 14512.01 6.08257755584916844 +AAAAAAAAPIIDAAAA English persons last there golden units. Special services help far vital fingers. Very complicated birds sho Home paint 0.74 1043.89 0.43753703896120444 +AAAAAAAAPKDDAAAA Hands might contact enough growing things. Criteria used to make convincing forms. Particular organizations sha Home paint 48.89 8562.98 3.58909551186812250 +AAAAAAAAACDAAAAA New, american owners might not appear. Parties move heavily also high variations. Unable, american terms might create indeed years. Nations absorb over normal experienc Home rugs 0.89 2701.48 0.99827241978850362 +AAAAAAAAAINBAAAA Concepts shall allow cautiously there Home rugs 4.82 8082.19 2.98659526203801105 +AAAAAAAABDJAAAAA Awards might mention better real, video-taped fires. Familiar patients must yield finally never net rules. Courses should attend; black ac Home rugs 0.79 120.11 0.04438400444970800 +AAAAAAAABHIAAAAA Smoothly main organisations yield here pensioners; subtle, british rights say public books. Only, social pairs take up to the police. Important, other men could go mor Home rugs 6.67 21599.16 7.98149374365127852 +AAAAAAAABNNBAAAA For example brief children must change almost. Fierce manufacturers ought to throw comfortably alone, subsequent loans; other boots switch. Very main men k Home rugs 7.88 1113.44 0.41144722266657961 +AAAAAAAABOGBAAAA Forms carry here american negotiations. Partly subject drivers should tell only stiffly local orders. Quite clean forces will enhance intentionally full ministers; stories mus Home rugs 7.64 9195.42 3.39796488383093785 +AAAAAAAACCFCAAAA Royal, comprehensive reports cost loyal, critical minutes. Exciting, short areas ought to pay for a appearances. Public, large institutions can Home rugs 4.30 2726.74 1.00760669630502701 +AAAAAAAACCHAAAAA Of course institutional forces occur violently from a governments. Patient, western teams count Home rugs 1.97 500.94 0.18511134117922509 +AAAAAAAACCPAAAAA Great images may not pay only, certain plans. Internationally new years command so in the days. Stairs tell teams; else unlike customers see elected, different numbe Home rugs 2.11 8294.23 3.06494997274915987 +AAAAAAAACDPCAAAA Organizations understand also instead accurate settlements. Costs become co Home rugs 7.44 12898.01 4.76617544944116470 +AAAAAAAACELAAAAA Broad, political premises must not continue even. Short local levels stay in a germans. Encouraging, poor priorities i Home rugs 9.98 13098.17 4.84014016787138328 +AAAAAAAACIGAAAAA Consumers must light now human schools; systems take Home rugs 37.18 2295.76 0.84834753189127999 +AAAAAAAADANDAAAA Hardly happy reforms may not try quickly along a pp.; sure sources use then now different Home rugs 3.58 2396.96 0.88574376243253759 +AAAAAAAADLCAAAAA However magic things should not take for a firms. Estimates supply; able, doubtful children must maintain left, lacking banks; simple sons c Home rugs 1.73 113.88 0.04208184519800805 +AAAAAAAADMFCAAAA Ideological members get sometimes modest abilities. Used, certain services would make all victorian, angry regulations. Even voluntary directions must sail however equations. Other, specific others ge Home rugs 8.46 4771.52 1.76321009834210907 +AAAAAAAADNGDAAAA Turkish members shall know to a subjects. No doubt decisive millions might know virtually public industries. Good, artificial Home rugs 1.62 4557.68 1.68419023728536476 +AAAAAAAAEDBAAAAA Softly social men get only with a miles. Only afraid difficulties should emerge t Home rugs 4.09 5355.01 1.97882597342628292 +AAAAAAAAEJFCAAAA Others could withdraw buildings. Clothes know partly. Inner prese Home rugs 4.44 7946.40 2.93641705902222677 +AAAAAAAAEJLBAAAA Parallel dead relations check further international men. Types improve apart half way steady ways; back metres shall not support at leas Home rugs 1.00 9684.36 3.57864188937285967 +AAAAAAAAELABAAAA Good, alone centuries might not see gently subjective ships. Less ambitious Home rugs 6.42 3762.17 1.39022704204943760 +AAAAAAAAFBPBAAAA Also other republics could not prescribe almost permanent mental p Home rugs 3.56 1252.71 0.46291138301718183 +AAAAAAAAFDIDAAAA Coastal agencies encourage. Obviously other events understand local students. Western subjects cannot set in a e Home rugs 6.19 3558.04 1.31479529757921118 +AAAAAAAAFICEAAAA Existing services make talks. Concerned, running Home rugs 30.02 2214.66 0.81837881354250538 +AAAAAAAAFOOBAAAA However major months execute either elements. Enough left provisions used to prove so away gastric police. Animals shall add faintly things. Well modern principles might pay suddenly other, soc Home rugs 1.32 16957.77 6.26637032001602569 +AAAAAAAAGBCBAAAA Mental horses could grab Home rugs 1.74 1044.31 0.38590175411601501 +AAAAAAAAGJOAAAAA Other, initial companies could know definitely mere funds. Italian years get already thereafte Home rugs 8.14 4357.37 1.61017008965967989 +AAAAAAAAHJCEAAAA Additional, interior police provide words. Different, long qualities answer really concerns; then other words state dry, political services. Awfully di Home rugs 9.78 7977.70 2.94798328447619281 +AAAAAAAAHLNDAAAA Firm, main skills can measure already electoral, white activities. Fairly disciplinary men protest there new changes. Strong, good reactions might prompt arbitrarily wild product Home rugs 6.42 9423.50 3.48224682317728204 +AAAAAAAAIBLCAAAA Origins used to play very on a matters. Long, important shows tackle more. Further vast fingers succeed only. Much dead values must rem Home rugs 4.71 7612.23 2.81293189736242391 +AAAAAAAAIEIAAAAA Possibly southern complaints would not produce to a years; months take. Services give; always professional days might develop quickly major urba Home rugs 36.03 10189.52 3.76531263858453641 +AAAAAAAAIHNBAAAA British stories ought to read furt Home rugs 2.05 1296.18 0.47897476386331293 +AAAAAAAAIKFCAAAA Better silent colleges protect never concessions. Certainly material words Home rugs 2.45 7108.50 2.62678957314752580 +AAAAAAAAINHAAAAA Still global systems would find real forces. Facts get rivals. Ahead british features must not rest nearly. Flats will restrict always subsequent miles. Then new children can allay only ordi Home rugs 8.72 430.95 0.15924807857465376 +AAAAAAAAINPBAAAA Possible Home rugs 0.41 9833.88 3.63389371141365844 +AAAAAAAAJEBCAAAA True Home rugs 55.56 1867.47 0.69008239771622846 +AAAAAAAAJINBAAAA Difficult writings improve full charges. Western incidents run in a options. Parts happen possible, forw Home rugs 4.45 2413.98 0.89203312847811273 +AAAAAAAAJKECAAAA Past losses will feel nowhere options. Political, free situations must produce selectively classes. Difficult ways believe sometimes enormous scientists. Interesting, simple rights ought to flush ago Home rugs 4.83 1972.51 0.72889761566142310 +AAAAAAAAJMMBAAAA Minds apply reluctantly dirty goods; therefore extended unions make secret, working men. Followin Home rugs 0.63 215.67 0.07969609724143306 +AAAAAAAAKAADAAAA Possible, false publications produce toda Home rugs 62.90 1868.41 0.69042975400781722 +AAAAAAAAKGMAAAAA Wonderful, scottish unions go nearby for a teams. Gladly current systems cannot look so major, emotional p Home rugs 7.31 5243.52 1.93762730007603777 +AAAAAAAAKLJCAAAA Dead names spend as a schools. Polit Home rugs 1.98 718.90 0.26565365747144353 +AAAAAAAALEGBAAAA Standard, foreign hospitals say later adult difficulties. Things ask very into a metals. Enough public persons will not give however ago sweet c Home rugs 0.57 5940.00 2.19499614046511968 +AAAAAAAALKFCAAAA Single institutions place also local ideas; variations used to appear yesterday domestic, corresponding attempts. Unlike, possible amounts open locally. National, main cig Home rugs 7.07 11038.74 4.07912318107709347 +AAAAAAAAMCLBAAAA Also noble characteristics might sound about a miles. Again social funds would stretch en Home rugs 7.90 2544.16 0.94013827958345773 +AAAAAAAAMFEBAAAA International metres minimise originally small allowances. Eminently favorite lines compare just never bottom things. British poets take countries; individual, in Home rugs 1.63 3135.51 1.15865864451006522 +AAAAAAAANBLDAAAA Colourful bones may adjust so. Pupils might catch so. Final, Home rugs 86.39 282.42 0.10436208922393251 +AAAAAAAANDFEAAAA Able armies bring certain, pretty requirements. Dogs pay weeks. Simi Home rugs 46.20 4674.82 1.72747674366484020 +AAAAAAAAOAACAAAA Foreign, absolute bills sh Home rugs 0.23 4232.41 1.56399387455656182 +AAAAAAAAOEPCAAAA Levels look only steep, cold results. Examples used to ensure together only expensi Home rugs 5.36 2875.57 1.06260354404668084 +AAAAAAAAOGBCAAAA African days incorporate economic, similar cells; vast, automatic stations ought to plan previously in a judges. Blank times would pay into the workers. Gradually ultima Home rugs 2.42 1831.70 0.67686438223736696 +AAAAAAAAOGEEAAAA Hands order. Pl Home rugs 91.05 5998.14 2.21648049662785404 +AAAAAAAAOGKBAAAA Magic facilities should not fight only likely months. Later present members absorb once more Home rugs 8.11 1193.91 0.44118313839439580 +AAAAAAAAOGLBAAAA As active accounts talk slowly. Big implications make as a children. Rounds should not check. Likely, military Home rugs 5.59 2607.00 0.96335941720413586 +AAAAAAAAONFDAAAA Prime members must need so regulations. Only injuries might run adequately to a shares; inevitably orthodox poets think yesterday protests. Thinking, full changes could put more. Months Home rugs 9.27 2740.60 1.01272835396611229 +AAAAAAAAPKJDAAAA Clinical photographs look also popular, common men. Loose, concerned earnings must go maybe only able enquiries; black unions observe exactly by a Home rugs 24.08 2749.12 1.01587673226859761 +AAAAAAAAACBEAAAA Directly green hours will maintain also Home tables 1.10 1433.48 0.74353680625971805 +AAAAAAAAAEDDAAAA Then legal services may bother circumstances; obvious, original years worry scottish, static areas; much fresh journals mean exactly routes. I Home tables 4.46 15267.45 7.91912758652365733 +AAAAAAAAAFFBAAAA Small motives shall use large, patient payments. Answers refer here odd, average officers. Always powerful sections might yield into a Home tables 4.41 5271.29 2.73418403568155059 +AAAAAAAAAGPBAAAA Odd, poor times could recycle suddenly eyes. Fa Home tables 0.33 2225.20 1.15419685052398680 +AAAAAAAAALACAAAA Perfect grants fight highly as great minutes. Severe, available millions like counties. Young legs cook however from a years. Early armed services reject yet with Home tables 4.31 7602.83 3.94353875654740364 +AAAAAAAAALJAAAAA True, particular parties drop for a times. Too mad Home tables 56.61 2020.10 1.04781280682343418 +AAAAAAAAANECAAAA Usually complete artists will give from the weeks. Units swallow political minutes; books might not arrest continually lips. Modest, royal problems must behave consequently genera Home tables 4.25 4496.26 2.33218098648974514 +AAAAAAAAAOPDAAAA Particularly popular detectives avoid rather free, major relations. Financial servants may know also widely surprising children. Delegates cannot get. However separate thousands discuss alway Home tables 4.93 7737.75 4.01352088807387150 +AAAAAAAACBODAAAA Nuclear needs can want. Overwhelmingly clo Home tables 0.43 930.32 0.48255096799365244 +AAAAAAAACDHBAAAA Enough bad rounds arrange later well black places. Courses reduce then in a experts. Also poor systems offer wonderful performances. Economic, unlikel Home tables 21.49 7678.11 3.98258600574183368 +AAAAAAAACFBDAAAA Actions see of course informal phrases. Markedly right men buy honest, additional stations. In order imaginative factors used to move human thanks. Centres shall catch altogether succe Home tables 1.61 33.06 0.01714800821423827 +AAAAAAAADNLBAAAA Federal, clear months please. New lips take slightly interesting Home tables 3.47 361.20 0.18735210426445445 +AAAAAAAAEEPDAAAA Roots should not lend overnight in a feet; fine children retire once usually evident forests. Sometimes novel effects might not go tons. Casualties involve more. Correct, perfect deleg Home tables 3.13 10251.08 5.31716890637669900 +AAAAAAAAEFEBAAAA Provincial, important tr Home tables 3.22 2399.31 1.24450658162444130 +AAAAAAAAEOFAAAAA Western, complex eyes can tell only regular acts. Perhaps high processes could put. Changes stay in the prisoners. Ages give now fascinating methods. British, quick words shall not expect new Home tables 4.27 9672.26 5.01693871537351095 +AAAAAAAAEPHAAAAA Now professional schools will not visit useful lists. Beautiful plans can recommen Home tables 2.52 408.50 0.21188630839432348 +AAAAAAAAFNJBAAAA Personal dimensions can dissolve final variations. Gradual sounds migh Home tables 1.19 5519.07 2.86270591938765946 +AAAAAAAAGBKDAAAA Hard sheets share so books. Permanent Home tables 31.00 443.40 0.22998871271001966 +AAAAAAAAGBMAAAAA Current degrees see in particular be Home tables 2.99 2250.99 1.16757395675039954 +AAAAAAAAGCDCAAAA Vast girls call benefits. Good, difficult makers deliver local things. High, formal hours play for a payments; well new men increase all equal newspapers. Top, total rights Home tables 2.62 10786.92 5.59510564931431049 +AAAAAAAAGHNAAAAA Just responsible poems ask only just joint patients. Solid, equal books prevent. Never universal fields must ignore black, main cameras Home tables 0.32 6835.22 3.54538441337343388 +AAAAAAAAGJHAAAAA Most official languages might not feel anywhere careful points; good, post-war prices refer originally ruling varieties. Increased lands would not get we Home tables 0.35 13164.59 6.82838770287595335 +AAAAAAAAGNHDAAAA Important, small girls should realise only high numbers. Previous, statutory products can give rather scientific methods. Isolated, living estates move now old trees; univ Home tables 2.85 3966.40 2.05734603088187185 +AAAAAAAAHFHDAAAA More german bags might not give always about a words. Recently new guests ought to Home tables 8.63 4805.11 2.49237948428065532 +AAAAAAAAIGDDAAAA Too labour operators tell more Home tables 3.43 9131.41 4.73640331783356027 +AAAAAAAAIIBDAAAA Families must not hear more great, english feelings. Proper faces justify extremely skills. Immediate discussions undertake often pa Home tables 0.18 2677.96 1.38904053470664016 +AAAAAAAAIODBAAAA Experts should not offer; low easy cities flourish particularly integrated, decisive Home tables 9.66 3549.82 1.84126867873766800 +AAAAAAAAIPJCAAAA Simply different statements complete always social, international speakers. Early serious buildings shall overcome just by a husbands; complex, common criteria will work little, fair countr Home tables 2.23 2835.45 1.47072957928196943 +AAAAAAAAJANDAAAA Only long brothers detect in a years; commitments can imagine near little great fields. Civil, soviet patients profit already just long arrangements. Often indi Home tables 8.94 690.05 0.35792447272338536 +AAAAAAAAJKHCAAAA Central houses increase actually essential payments. Minor organizations take subsequently careful players; good, molecular righ Home tables 7.94 13582.01 7.04490075758821408 +AAAAAAAAKFABAAAA Women get also chairs. Full, integrated paintings sit Home tables 6.34 1123.11 0.58254989429803830 +AAAAAAAALJOAAAAA Wild volunteers expand approximately sales. Specific, close versions must stress longer able powers. Far me Home tables 3.86 2363.26 1.22580767974533392 +AAAAAAAAMEKDAAAA Bold parties could revert newly equal plans. Also other products cry as at least lovely discussions. Manufacturing, french letters lay economically ready duties; serious, stron Home tables 1.02 2741.71 1.42210724746095625 +AAAAAAAAMGGBAAAA Areas ought to calculate slowly charges. Difficult, national participants might not write away bus Home tables 4.21 5457.26 2.83064547208814138 +AAAAAAAAMLKCAAAA Closely young offic Home tables 8.10 25.92 0.01344453638575487 +AAAAAAAANAIBAAAA Wide, new changes reduce highly on a notes. Nurses re Home tables 0.25 1860.34 0.96494632792728456 +AAAAAAAANFABAAAA Critical, neighbouring feelings should achieve unusual, hungry types; po Home tables 5.93 619.20 0.32117503588192191 +AAAAAAAANJNCAAAA A Home tables 4.83 2031.72 1.05384002568155423 +AAAAAAAANOJBAAAA New situations seem. National missiles will cater departments. Women come astonishingly. Spanish mont Home tables 5.87 8171.71 4.23861313382858538 +AAAAAAAAODODAAAA Highly tory votes could no Home tables 8.80 3686.85 1.91234525361961205 +AAAAAAAAOGPCAAAA Slight, present techniques run writers. Schemes make. Grand boys could help fine, past re Home tables 1.51 332.04 0.17222700083048022 +AAAAAAAAONNDAAAA Dead, big talks will rest old offers. Dead, competitive authorities occupy alone Home tables 0.38 2425.28 1.25797705268686622 +AAAAAAAAPDGEAAAA Almost working things shall not see import Home tables 3.78 3316.68 1.72034046836055031 +AAAAAAAAPHDAAAAA Police know more families. Atlantic birds might keep there far presen Home tables 40.62 0.00 0.00000000000000000 +AAAAAAAAPHDDAAAA Obviously elaborate members would not retu Home tables 3.94 610.39 0.31660534585265877 +AAAAAAAAPLFDAAAA Quiet levels must achieve. Local, national metres fill to a businessmen. Real, key boots could not determine at best. Young groups may know ever happy, magnetic difficulties Home tables 2.15 NULL NULL +AAAAAAAAAMODAAAA Labour, middle children might produce useful signals. Surprising farmers kill on the costs. Trees return recent, single animals. Original governments read over there. Previous Home wallpaper 3.08 5699.40 1.39109945794862842 +AAAAAAAAAPGCAAAA Once again only measures shall destroy independent, normal prisons. Present, industrial ambitions can prevent as employers. Large, previous origins say inside Home wallpaper 3.32 262.60 0.06409494291632625 +AAAAAAAABBHBAAAA Reports can say. Constant, other keys will analyse here white months. Dreams would not change to a neighbours; visual, financial wages set in a girls. Fingers Home wallpaper 4.24 9127.17 2.22774348871898495 +AAAAAAAABCPBAAAA Nearer regular men like in a ministers; children come therefore female views. Only financial events must not allow old miles. Very british forces get. Home wallpaper 9.72 5545.66 1.35357487103333520 +AAAAAAAABPNCAAAA Great, strategic live Home wallpaper 2.35 12111.89 2.95624866016307208 +AAAAAAAACCOBAAAA Groups can consent close. Awful, soft friends pursue comfortable departments. C Home wallpaper 6.57 1777.90 0.43394668320996359 +AAAAAAAACDBEAAAA Empty, additional russians should ensure commonly in a books. Sure, close difficulties follow always on a weeks. Royal y Home wallpaper 0.85 328.29 0.08012844177456491 +AAAAAAAACFPBAAAA Educational, reasonable rooms mi Home wallpaper 2.73 737.08 0.17990518097778275 +AAAAAAAACGHCAAAA Then french ministers aid Home wallpaper 3.16 7027.37 1.71522802361730232 +AAAAAAAACHCDAAAA Old eyes would not develop to a parents; nice, red games come now to a molecules. Sheer centuries could follow as usually late powers; backs affect police. Almost tiny trees shall buy fro Home wallpaper 1.22 20810.71 5.07944123952101991 +AAAAAAAACKBDAAAA American, long organisations terminate for a agents. Facilities determine open. Now general students rebuild even particular pounds. Good teachers might not press names. Guidelines evaluate clear Home wallpaper 4.09 293.44 0.07162231549644621 +AAAAAAAACNCCAAAA Public Home wallpaper 0.64 1015.94 0.24796883589646797 +AAAAAAAACNICAAAA Initial unions agree still otherwise individual councillors. Leading minutes bring mathematical conditions. Full, huge banks must not feel exclusively special lines. Ago other cases will hold Home wallpaper 8.36 1699.28 0.41475725285169409 +AAAAAAAACNOCAAAA Fresh, othe Home wallpaper 8.40 501.78 0.12247357371117359 +AAAAAAAADBGDAAAA Ahead national cir Home wallpaper 14.29 13998.80 3.41680231110840781 +AAAAAAAAEDDBAAAA Still fortun Home wallpaper 4.83 4391.94 1.07197693675525478 +AAAAAAAAEDKDAAAA Minor, single things could cry too profits. Examples focus material, young observations. Existing tensions would stop away. Facilities reply most thoroughly small Home wallpaper 3.85 6735.50 1.64398891094027208 +AAAAAAAAEEPCAAAA Wooden, clear considerations will not become now proceedings. A bit institutional firms will Home wallpaper 4.94 9408.96 2.29652229284842735 +AAAAAAAAEKDCAAAA Thick, other ways come completely. Careful men would find later there valid children. Interesting owners allow a bit best wide polls. Miles behave other, considerable heads; inte Home wallpaper 0.96 3860.39 0.94223715416891351 +AAAAAAAAEKECAAAA Marked, free flowers carry restrict Home wallpaper 0.67 4918.41 1.20047680193864503 +AAAAAAAAFEKDAAAA Less western books give physically only Home wallpaper 4.22 5084.28 1.24096205777082719 +AAAAAAAAFJACAAAA NULL Home wallpaper NULL 15833.49 3.86461019693915650 +AAAAAAAAGCIDAAAA Liable, other others provide also in a resources. Months get briefly long sheets. Windows talk activities. American Home wallpaper 5.42 151.36 0.03694368073044608 +AAAAAAAAGDIDAAAA New citiz Home wallpaper 3.50 6508.22 1.58851481106966039 +AAAAAAAAGGABAAAA Main elements write generally however secondary periods. Documents persuade empty, labour margins. Over other friends contend afterwards friendly, labour buildings. Canadian birds Home wallpaper 4.07 2883.10 0.70370194182048822 +AAAAAAAAGHAEAAAA Shortly economic records cause nevertheless by a requirements. Privately silent forms take. Pink leaves aba Home wallpaper 8.70 0.00 0.00000000000000000 +AAAAAAAAGKADAAAA Stores visit values. Others cannot hang around rather civil brothers. Direct systems go then free, other instructions. Difficult, top feet will al Home wallpaper 13.91 2088.96 0.50986965710010998 +AAAAAAAAGLJCAAAA Small, social patterns design deeply without a judges. Moving feet arrange in the developments; sports say Home wallpaper 0.63 13980.62 3.41236496890650830 +AAAAAAAAGMMCAAAA Tests should allow finally times. Thus other differences act already important weapons. So ridiculous spor Home wallpaper 3.26 12082.76 2.94913866135441792 +AAAAAAAAHEGCAAAA Courts must not understand ideas. British figures would isolate sure young preparations; able, short governments should express more private properties. Countries de Home wallpaper 0.28 15297.35 3.73375009528203862 +AAAAAAAAHKHBAAAA Military, poor questions challenge that with a costs. Appropriate, main patients will not see concerned, industrial findings; terrible, concerned eyes decl Home wallpaper 3.37 3242.71 0.79147491372505823 +AAAAAAAAIALAAAAA Green, european terms privatise new arms; also local duties need damp, successful professionals. Fresh, furious odds will undertake too only probable players. Home wallpaper 2.81 227.73 0.05558393507362900 +AAAAAAAAICBCAAAA Impossible, other patients provide somewhat. Initially helpful ref Home wallpaper 2.44 10361.84 2.52909955562873563 +AAAAAAAAIDFEAAAA Always western women run soon in the solutions; left members should allow national, innocent products. Of course left tonnes thank unduly especially interested customers. Elderly pen Home wallpaper 0.99 7449.54 1.81827052952356833 +AAAAAAAAIEOCAAAA Artificial, careful years behave even specialist volumes. Assistant che Home wallpaper 7.43 6528.95 1.59357455275532468 +AAAAAAAAIGLBAAAA Short things come from a activities. Losses should not work ro Home wallpaper 9.19 3438.64 0.83929716111879700 +AAAAAAAAILHCAAAA Courts can pu Home wallpaper 9.63 7132.45 1.74087576391298992 +AAAAAAAAINEBAAAA Representative, keen problems might exam Home wallpaper 6.78 17424.37 4.25290936977512414 +AAAAAAAAIPFEAAAA Useful developments might control effective, unknown homes. Other, right marks cannot become by the moments. Natural, christian bars used to enable also new Home wallpaper 75.10 6730.56 1.64278316448937089 +AAAAAAAAJCCAAAAA Perhaps different figures hang new women. Dynamic goods finance now; birds keep already proposals. Schemes guess animal Home wallpaper 4.93 11316.14 2.76202340949412078 +AAAAAAAAJGBDAAAA S Home wallpaper 2.23 2663.69 0.65014873761153490 +AAAAAAAAKAMDAAAA Difficulties should Home wallpaper 3.85 3734.34 0.91147109341261905 +AAAAAAAAKCCAAAAA New, poor adults used to fear; new offers may make undoubtedly cells. Clinical dogs decide. Then poor models know then entirely rea Home wallpaper 0.20 10778.60 2.63082159831650459 +AAAAAAAAKGHCAAAA Significantly poor employees will not attend over interactions. Other babies used to choose departments. Young members repair. Easy patients find ever pers Home wallpaper 6.87 6138.42 1.49825468201232053 +AAAAAAAAKKFBAAAA Perfectly tall bodies think there a Home wallpaper 6.25 2518.24 0.61464755921404955 +AAAAAAAALCKCAAAA Areas would stop also logical, local initiatives. Existing, increasing words should take open concerns. Objectives protect jointly at t Home wallpaper 6.48 7065.22 1.72446638458220312 +AAAAAAAALIGDAAAA Human, back b Home wallpaper 4.28 8161.86 1.99213233355310951 +AAAAAAAAMAMCAAAA Measures should make rec Home wallpaper 2.45 3471.50 0.84731757172135024 +AAAAAAAAMBAAAAAA Familiar thanks should see proposals; more single lakes shall not announce employees. Specified lawyers canno Home wallpaper 7.89 509.65 0.12439446937283196 +AAAAAAAAMDDBAAAA Basic moves mig Home wallpaper 0.30 11860.26 2.89483125541807904 +AAAAAAAAMDDEAAAA Components could identify hopef Home wallpaper 1.39 1204.56 0.29400687143674770 +AAAAAAAAMDEDAAAA Social dealers shall emerge even figures. Clear prayers could not send. Home wallpaper 6.93 6706.36 1.63687647134932864 +AAAAAAAAMDEEAAAA Actual, urban police learn quickly low, british years; ethnic, common months should fail then overall markets. Years get. Criminal statio Home wallpaper 7.74 1379.50 0.33670591680530107 +AAAAAAAAMEMAAAAA Particularly tight problems cannot lead special, simple sales. Warm bodies get. New, primary attempts wo Home wallpaper 5.23 15517.89 3.78757910788967986 +AAAAAAAAMHKDAAAA Chief, other others speak fairly; established years may reduce political museums. Vulnerable, male features sug Home wallpaper 4.79 7653.42 1.86803319883727966 +AAAAAAAAMIPDAAAA Much following charges cannot complete difficult, effective jews. Poor, commercial pro Home wallpaper 1.85 5730.05 1.39858045566525218 +AAAAAAAAMMLCAAAA Special, long-term cases may not like sharply favorite arms. Insufficient papers bring. Legal cheeks could not apply with a sales. Terms give. Judicial, natural sets see at the cells. Home wallpaper 2.40 15153.09 3.69853936997697683 +AAAAAAAAOAJDAAAA Sensitive, labour areas would not suffer general, successful seconds; golden, substantial methods pay then available beliefs. Afterwards round years will defeat Home wallpaper 1.96 4949.14 1.20797732591358298 +AAAAAAAAOCKDAAAA That positive banks ought to fall perhaps eligible, white proceedings. Voluntary, political bodies suggest united, unlikely women; soviet, long comm Home wallpaper 5.69 NULL NULL +AAAAAAAAODACAAAA Later recent years could take further; opening intervals weaken; personal years say often. Main pairs generalize articles; functions know quite other varieties. Pounds include to the hands. Claims h Home wallpaper 1.19 7033.67 1.71676571645954473 +AAAAAAAAODCEAAAA Long potential cards make previous subjects. Continued, firm rounds might support. Royal, powerful vessels exist employees Home wallpaper 1.91 7286.37 1.77844428490949006 +AAAAAAAAOGIDAAAA Societies could make now below a lev Home wallpaper 6.61 5369.24 1.31051458988596934 +AAAAAAAAOGPDAAAA Boxes would not let further level groups. Different priests get chapters. Languages may stop still legs. Blocks must make good, important securities. Complete diffe Home wallpaper 4.83 1053.00 0.25701437506051615 +AAAAAAAAOMBEAAAA Protective, absolute fingers could hear usually daily, rapid schemes. Normal y Home wallpaper 6.16 437.24 0.10672076481620141 +AAAAAAAAONGBAAAA Brown, natural periods might avoid in a changes; standard, military improvements should seem enough. Things commit easily from a hopes. General courts could close part Home wallpaper 2.54 1591.79 0.38852128402429154 +AAAAAAAAOOMBAAAA Times used to remember to the trains. Evidently chief tests will not look often apparent foreign holidays. Images will not meet earlier rows. Today happy months cannot like as against th Home wallpaper 5.03 5511.22 1.34516881682907673 +AAAAAAAAPCIDAAAA Proteins must remember above beneath available rights; good pl Home wallpaper 0.82 8210.81 2.00407996285910406 +AAAAAAAAADKDAAAA No equal occasions care poor materials. Students could not operate briefly shares. Very determined effects go already heavy factors; full possibilities make certainly by the posi Sports archery 6.40 8728.20 2.57886262177629984 +AAAAAAAAAEJBAAAA Appointments will not go inc, temporary factors. Static, specific proport Sports archery 1.85 1021.30 0.30175665035404036 +AAAAAAAAAMIAAAAA Lives shall mean servants. Short inner balls shall take policies. Sports archery 0.82 20373.51 6.01962413938563079 +AAAAAAAABAFBAAAA Eyes can go extremely hard numbers. Early, real others would discuss also. Good members Sports archery 4.61 3215.40 0.95003263835149453 +AAAAAAAACDFDAAAA Days can establish routine members. Associations replace both simple, crucial areas. Parties transmit variables. Statistical foreigners should not play Sports archery 2.48 2613.03 0.77205442090925102 +AAAAAAAACECCAAAA Players will come just about senior matters; external hours may become natural principles. Smooth, national sentences can support public women. Protests tell too in a leaders. Labour studi Sports archery 1.36 426.80 0.12610372894458477 +AAAAAAAACIIBAAAA Just silver assets move quite both statistical difficulties. Mainly national hours must prevent. Electronic Sports archery 9.78 10843.65 3.20390042260999677 +AAAAAAAADGNDAAAA Entirely social buildings step all including the standards. Massive months read children; irish years come for a words. Sports archery 5.76 12915.10 3.81593783901641692 +AAAAAAAAEAEDAAAA Religious, subsequent views cannot meet around important min Sports archery 5.76 23175.78 6.84759203186346949 +AAAAAAAAEFIAAAAA Shares take. Consequences warn liberal, fresh workshops; illustrations ought to measure sports. White, universal organizations assist young provisions Sports archery 5.83 3736.19 1.10390696121243713 +AAAAAAAAEHDDAAAA Long, immediate cars Sports archery 0.47 7961.21 2.35224523877909490 +AAAAAAAAEHGCAAAA Holy days like new years. Excellent, standard standards regain more simply friendly others. Easily previous texts can Sports archery 1.24 2736.34 0.80848799826669420 +AAAAAAAAEIEBAAAA Low days go photographs; attacks may not tear probably similar, mathematical police. Likely, small name Sports archery 2.59 11492.70 3.39567086607645118 +AAAAAAAAEMADAAAA Now public weapons used to specialise always limited Sports archery 6.16 609.03 0.17994600290328131 +AAAAAAAAFADCAAAA Materials go furt Sports archery 3.67 48.41 0.01430337750282884 +AAAAAAAAFHBAAAAA Previously white patients should set sometimes level theoretical studies. Federal, european trends keep. Social, other hills can leave opportunities. Organisers lower experiences. Recent criteri Sports archery 2.18 4063.94 1.20074505203152723 +AAAAAAAAFLFBAAAA Scientific, elegant blues must eliminate. Basically local musicians might slow never now spiritual bedrooms. Wrong studies ought to impose relations. S Sports archery 1.70 4653.68 1.37499156821657742 +AAAAAAAAGAKDAAAA Constant, olympic languages could not bow other Sports archery 2.01 7616.46 2.25038427215855694 +AAAAAAAAGDNAAAAA Strong, essential rates could Sports archery 8.43 4002.55 1.18260656112265174 +AAAAAAAAGEMDAAAA Critical, secondary cars will extend social parts; together serious voices see personally a Sports archery 42.19 29.70 0.00877525948841183 +AAAAAAAAGNGBAAAA Women aim entirely reasonable, whole surfaces. Young drawings meet then sure, executive projects. Public, new offers used to sweep too light, old ar Sports archery 65.59 3949.47 1.16692337009083694 +AAAAAAAAHADDAAAA Marginal, bright boats re-open subsequent figures. Most anxious positions produce nearly together with a causes. Invariably necessary hands must not le Sports archery 8.66 312.64 0.09237364062145029 +AAAAAAAAHCAEAAAA So blue parents will get at a conferences. Toxic methods Sports archery 1.14 2037.09 0.60188529802184673 +AAAAAAAAHCEAAAAA Differences give financial, current reasons. Working, legal memories cannot launch into a activities; small, difficult parties coul Sports archery 1.62 7284.54 2.15231409945169992 +AAAAAAAAHFAEAAAA Competitive holidays should not keep all democratic, o Sports archery 61.08 8753.34 2.58629056869679390 +AAAAAAAAHKMCAAAA Crude, silly estates walk. Specific eyes mus Sports archery 3.16 11104.29 3.28090997254466541 +AAAAAAAAHMKCAAAA Normally eastern men protect also only explicit quantities. Royal, modest miles build by a opportunities. Shoulders judge more slightl Sports archery 5.58 12487.62 3.68963319503977423 +AAAAAAAAICGBAAAA Nowhere other groups say home chief members. Contemporary letters practi Sports archery 8.43 2359.96 0.69728152802263887 +AAAAAAAAIHCAAAAA Current children take additional workers; far waiting arguments like bad, little days. Comp Sports archery 2.50 7478.91 2.20974329765919510 +AAAAAAAAIIDBAAAA Aware families give men. All social winners pose confident, new increases; most glad years wonder in genera Sports archery 1.55 2973.81 0.87865166394727186 +AAAAAAAAIJGAAAAA Welcome, united enemies fall. Nationally late profits complete mili Sports archery 7.03 11118.64 3.28514987064765227 +AAAAAAAAILDDAAAA French photographs shall not advise clearly for an demands. Important, statutory cases rate well times. Other, local doctors assess terms. Normally white considerati Sports archery 7.09 408.72 0.12076175279810376 +AAAAAAAAILFBAAAA Designs would throw especially linear, horizontal characters. Fundament Sports archery 3.73 8691.82 2.56811366756120145 +AAAAAAAAILPAAAAA Changes set shortly. Mental, different jobs need more with a solicitors. Other, federal pieces thank then to a chang Sports archery 1.50 15462.27 4.56853304814429410 +AAAAAAAAIMJDAAAA Other consequences may shape quite. Personal, particular lawyers take brown, large men. Skills would gather as busy fears. Days will Sports archery 3.96 12677.69 3.74579190113278554 +AAAAAAAAIPADAAAA Political troops forget plates. Emotional lists must intervene virtually in the children. Ready, only Sports archery 7.31 402.50 0.11892397118133873 +AAAAAAAAJDNAAAAA Months could not change curiously public contexts. Confident hotels would motivate in a studies. Workers sing fully again due positions. Irrelevant hands might create otherwise here strategic po Sports archery 0.40 1385.73 0.40943233437296029 +AAAAAAAAJJNDAAAA In short major reasons ought to sell already professional local institutions; corporate, able jobs will insure so su Sports archery 9.22 989.95 0.29249387644960565 +AAAAAAAAKAECAAAA Privileges face mostly solicitors. Different soldiers suggest home. Deep stations make right parents. Safe, central things would tackle just. As famil Sports archery 37.12 16530.14 4.88404942356147718 +AAAAAAAAKGCBAAAA Goods go only. Accountants may unite. Almost agricultural muscles go just regional police. Real samples used to build auditors; following women can believe. Very concerned tonnes would fit there Sports archery 7.66 2295.32 0.67818278144583953 +AAAAAAAAKIPCAAAA Young countries should restore increasingly others. Combined, large activities match in a cases. Positions can Sports archery 4.34 2791.69 0.82484189094964351 +AAAAAAAAKJEDAAAA Local, main troops cannot support never diffe Sports archery 3.65 463.60 0.13697677773830717 +AAAAAAAAKKHBAAAA Earlier controversial auditors s Sports archery 2.90 258.93 0.07650430772169947 +AAAAAAAAKNBEAAAA Old relationships in Sports archery 0.71 2104.62 0.62183793348489221 +AAAAAAAAKPABAAAA Individual, grand relatives must provide much areas. Italian, respectable experts might revise nationally public standards. Comfortable forces record forward importan Sports archery 3.59 7433.10 2.19620812469070534 +AAAAAAAAMCKBAAAA Patient teachers shall stop already serious weeks Sports archery 2.66 11143.58 3.29251872491165869 +AAAAAAAAMGEAAAAA Schools will get financial, small years. Chronic, real societies Sports archery 93.67 840.45 0.24832211572510841 +AAAAAAAAMIFEAAAA More leading requirements cross; elderly, able structures know obviously only houses. Enough light populations postpone blank payment Sports archery 2.76 5506.32 1.62691538135460637 +AAAAAAAANDOBAAAA Real pupils could adopt fine years. Big neighbours must understand for a visitors. Duties would not give almost at last blue priests. Previous, small miles make finally Sports archery 7.47 1309.14 0.38680280157102555 +AAAAAAAANKJCAAAA Domestic, chief devices can capture provincial lin Sports archery 3.78 18988.01 5.61025976156763126 +AAAAAAAANOHAAAAA Strings ought to include even. Difficult, medical Sports archery 64.26 5845.14 1.72702425071028634 +AAAAAAAAOAGAAAAA Big affa Sports archery 7.86 4365.75 1.28991882530417280 +AAAAAAAAOILBAAAA There aware passengers allow all after a reservations. Simply environmental feet may close hardly labour members. Influential, old shareholders must Sports archery 2.48 5434.29 1.60563316112058941 +AAAAAAAAOKNBAAAA Bad publications improve by the years. Regular movements might give at least profits. Hard tests might not meet Sports archery 9.45 12999.48 3.84086903078854452 +AAAAAAAAAGCAAAAA Sources make visual representatives. European regions will not run unacceptable loans. Right, natural firms get merely moral buildings. Virtually various sa Sports athletic shoes 2.23 3212.86 1.46013319558188889 +AAAAAAAAAJLCAAAA Distinguished powers upset very at a makers; animals shall see afterwards complete, working institutions. Sports athletic shoes 4.30 909.15 0.41317707424639551 +AAAAAAAAAKEDAAAA Seriously social measures might give. Less technical travellers contradict entirely for a possibilities. Major, young police give only; more than important findings be Sports athletic shoes 35.35 15716.62 7.14265750276894310 +AAAAAAAAAOECAAAA Priorities jump now important drawings. Both still movements will determine early massive, right patients. As huge goods might include at least chi Sports athletic shoes 1.75 11184.41 5.08292559090593238 +AAAAAAAABCCCAAAA Degrees know as after a heads; new, complex ma Sports athletic shoes 1.41 3007.89 1.36698145504591167 +AAAAAAAACCFEAAAA Real, comparative methods insta Sports athletic shoes 1.70 11493.02 5.22317810906375025 +AAAAAAAACFNDAAAA Develop Sports athletic shoes 6.28 2742.72 1.24647090697582786 +AAAAAAAADOGCAAAA However local things might not feel regional, responsible roots. Local, suitable nations set strong days. Patients might seem to a rooms. Sure othe Sports athletic shoes 2.00 303.48 0.13792111146928022 +AAAAAAAAEBIDAAAA Enormous, pure beaches lie highly financial periods. So united ships used to stay. Simply famous tons shall ensure separately extensive needs. In order educational statements must not pa Sports athletic shoes 3.52 3499.90 1.59058289848205428 +AAAAAAAAECBDAAAA Grey problems must not acquire detailed times. Sports athletic shoes 16.36 1039.15 0.47225755563233998 +AAAAAAAAECBEAAAA Current, political advantages will g Sports athletic shoes 3.15 125.13 0.05686723566017871 +AAAAAAAAEGLCAAAA Prices ought to go yesterday. Interests might rest here funds. Letters damage also rich agreements. Central, i Sports athletic shoes 1.72 128.63 0.05845786400518490 +AAAAAAAAELIAAAAA Generally top practices can reduce most links. Earnings will tell as techniques. Flat direct measures would not go far material whole sentences. Simply defensive services evaluate nat Sports athletic shoes 6.06 794.64 0.36113625945020704 +AAAAAAAAELNBAAAA Sentences will retire always from the marks. Modern activities may perform lon Sports athletic shoes 4.66 1180.16 0.53634169932643252 +AAAAAAAAENFEAAAA Almost uncomfortable shares may believe wrongly constant levels. Red, other words used to resist more frien Sports athletic shoes 0.12 23738.70 10.78841402674246177 +AAAAAAAAFHMBAAAA Items used to thin Sports athletic shoes 4.26 23.25 0.01056631686325545 +AAAAAAAAGNCCAAAA Eyes may not give children. Good great beans shall cook. Visible, Sports athletic shoes 36.86 5204.23 2.36514164340902922 +AAAAAAAAGNPAAAAA Religious, alone results go all investigations. Banks ma Sports athletic shoes 1.04 3489.00 1.58562922735046355 +AAAAAAAAHFNDAAAA Homes cannot inform almost fresh hotels. Plans could kill today hi Sports athletic shoes 3.62 7136.25 3.24317757915727874 +AAAAAAAAHJFEAAAA Woods wear indeed from a numbers. Counties must not receive as greatly public windows. Above hostile groups breed of course usually true members. Sources introduce similarly words. Largel Sports athletic shoes 8.59 4113.45 1.86942004736164067 +AAAAAAAAHOFCAAAA Military, considerable sizes wash social consultants. Equal ways stand detailed writings. Tough, potential directions interpret then. Free wives would restore still. Better fresh men carry others. St Sports athletic shoes 8.09 4091.45 1.85942181205017314 +AAAAAAAAIGKCAAAA As usual religious variables may believe heavy, available sister Sports athletic shoes 6.51 590.67 0.26843898415566016 +AAAAAAAAIIFEAAAA Objectives shall get with a years. Huma Sports athletic shoes 6.42 6968.96 3.16715008891839681 +AAAAAAAAJFCDAAAA Existing theories wait supplies. Proper partners measure things. Areas must not thank a little. Hard white rules formulate then major, institutional differences. Sports athletic shoes 1.47 16050.71 7.29448979527840609 +AAAAAAAAJLCAAAAA Absolute companies might not raise in order powerful, recent waves. Major chil Sports athletic shoes 0.18 14627.31 6.64760397062645716 +AAAAAAAAKAJBAAAA NULL Sports athletic shoes 0.74 2201.76 1.00062338997167000 +AAAAAAAAKAOBAAAA Clean, large conditions would understand labour dates. Large clergy should give high jobs. Patients might escape. As national polic Sports athletic shoes 5.50 257.64 0.11708842480211334 +AAAAAAAAKCAEAAAA Particular, financial years shape then twice visual friends. Limited, future women ought to come casual scots. Relations concentrate charges. Shares shall establish in a plants. Then double Sports athletic shoes 4.22 164.92 0.07495040761669202 +AAAAAAAAKGKDAAAA Presumably yo Sports athletic shoes 4.44 163.80 0.07444140654629003 +AAAAAAAAKIBBAAAA In particular financial studies can gain less than huge, model consequences. Really other activities walk o Sports athletic shoes 47.58 1719.85 0.78161204547397384 +AAAAAAAAKIEDAAAA Now political women could Sports athletic shoes 8.57 57.80 0.02626809095467377 +AAAAAAAAKIFCAAAA Chronic lines shall take enough by the sales; international, welsh angles used to rule now front powers. Standard othe Sports athletic shoes 3.00 16781.46 7.62659027045362857 +AAAAAAAAKLABAAAA Skills use rather than a principles. Easy employe Sports athletic shoes 6.29 9250.24 4.20391255488860762 +AAAAAAAAKLNDAAAA Accounts could think aspects. Industrial, large Sports athletic shoes 1.92 6322.30 2.87326559589505180 +AAAAAAAAKMECAAAA Cells call no doubt pilots. Arms should pay rather good duties. Thus long s Sports athletic shoes 9.73 857.50 0.38970394452651834 +AAAAAAAAMDHAAAAA Friends cry easily sure varieties. Appropriate proposals provide recently between a books. New, considerable forces seem like the elections. Right big clothes fr Sports athletic shoes 9.64 2708.86 1.23108271390099647 +AAAAAAAAMPHBAAAA Words live only anxious countries. British, northern substances criticise most extra, Sports athletic shoes 3.18 2390.50 1.08639915963923277 +AAAAAAAANCCCAAAA New rules continue wet cuts. German, following procedures shall see findings. As good charges cannot pay notably routine, short plates. Problems used to alleviate a Sports athletic shoes 30.73 3030.00 1.37702968153393653 +AAAAAAAANCNAAAAA Supposedly parental instructions see. Broken, raw habits should not issue at all friendly beliefs. Certain constraints know Sports athletic shoes 0.59 5983.42 2.71925641487913747 +AAAAAAAANGOBAAAA Also other measurements pay at least around the artists. Perfect, good cul Sports athletic shoes 2.83 4854.06 2.20600154981736633 +AAAAAAAANLBAAAAA Democratic forests use on a communities. Potatoes could not include still easy movies. Direct leads could sh Sports athletic shoes 3.61 1739.94 0.79074225217430942 +AAAAAAAAOAPCAAAA Levels may not go involved issues. Miles will beat good institutions. Tiny, c Sports athletic shoes 9.51 9805.35 4.45619075505900481 +AAAAAAAAPCODAAAA Never national communities could turn so. National, whole styles buy far really high leaders. Indeed beautiful others liv Sports athletic shoes 5.39 306.50 0.13929359649839985 +AAAAAAAAPDABAAAA More than hot women govern only full polic Sports athletic shoes 1.64 3354.48 1.52449456307325393 +AAAAAAAAPIFAAAAA Notably international minutes write too national, important visits. Human, clean patients Sports athletic shoes 1.21 6716.71 3.05251123176759302 +AAAAAAAAAJIDAAAA Major missiles may reply british dogs. Other, c Sports baseball 1.15 12361.94 4.21788969172030922 +AAAAAAAAALEBAAAA Also other adults ought to uphold usually in a hills; carefully good signs would ensure with an benefits. Continuous, nuclear days shall feel just in the politicia Sports baseball 0.75 3265.70 1.11425572088612417 +AAAAAAAAANCBAAAA Therefore unexp Sports baseball 3.99 3063.58 1.04529244615007878 +AAAAAAAAAOMCAAAA Often unnec Sports baseball 6.08 2524.58 0.86138583085852692 +AAAAAAAABDOBAAAA Eggs shall not encourage as. Economic classes act other girls. Technical features wash even. Social goods can monitor probably Sports baseball 2.18 3658.98 1.24844272211406762 +AAAAAAAACBGCAAAA Managers shall put even. Physically additional guests help therefore high times; here specialist successes tend old plan Sports baseball 9.08 251.02 0.08564793797863701 +AAAAAAAACGMAAAAA Dreams cannot need further at a securities. Modern societies devote once again for a businesses; ways used to say to a Sports baseball 1.06 4758.65 1.62364974927113782 +AAAAAAAACJJCAAAA Fun activities cost white camps. Bare, solar databases go especially days. More subject sites deal certainly; partly equal occasions hear subs Sports baseball 6.89 1014.60 0.34618117230947778 +AAAAAAAADDLCAAAA Most other delegates enhance natural, successful shows. American, similar times can derive easy, small departments. Artificial, other manager Sports baseball 4.91 1022.10 0.34874016973932312 +AAAAAAAAEHFAAAAA Fully silent bishops ought to seek only. Just new forms change immediately deeply raw cells. White corners shall lighten really reportedly glad games; teachers think at pre Sports baseball 3.06 14501.24 4.94781811860939439 +AAAAAAAAEHFEAAAA Winds owe urgently military managers. Internal conditions used to satisfy now as disable Sports baseball 7.10 7772.75 2.65205963637738361 +AAAAAAAAEHNBAAAA Organisations restore far. Far notes might not ask very places. Innocent requirements would not change to a children. Cer Sports baseball 1.20 8146.44 2.77956253631857102 +AAAAAAAAEHOAAAAA Also international soldiers use shortly decisive parties. Major, above advertisements expect about already loyal stairs. Lucky, small towns appear. Then english children corresp Sports baseball 1.92 3722.51 1.27011913634314422 +AAAAAAAAEIKDAAAA Guilty, oth Sports baseball 3.01 5530.46 1.88699105678166221 +AAAAAAAAFOIBAAAA Rather american exercises might remember times. Below western accidents give Sports baseball 0.71 7321.35 2.49804211106642533 +AAAAAAAAFOMDAAAA Later federal objectives Sports baseball 5.97 7447.00 2.54091384800776761 +AAAAAAAAGFKCAAAA Feet used to make import Sports baseball 2.92 798.30 0.27237968643273813 +AAAAAAAAGKHCAAAA Parents induce free deaths. Empty, red rec Sports baseball 39.45 15343.37 5.23515258602214870 +AAAAAAAAGKMAAAAA Symbols could enable too wrong problems. Real, old Sports baseball 0.29 5569.42 1.90028419543056548 +AAAAAAAAGLNCAAAA Elements shall arrange more. Coins would constitute however. Departments subscribe only in a children. And so on significant areas protect within Sports baseball 1.17 1171.52 0.39972222253498857 +AAAAAAAAHDFBAAAA Residents will happen here police. Owners might not match lines. Temporary, good symptoms used to achieve about in a issues. Troops can arrange. Even true comments shall not get ba Sports baseball 3.86 3886.24 1.32598375623495459 +AAAAAAAAHMMAAAAA Relevant numbers happen by the variables. Basic, italian fingers l Sports baseball 8.19 5295.33 1.80676478135772420 +AAAAAAAAHNDCAAAA Fascinating companies could tell partly about a Sports baseball 8.54 2203.05 0.75167990504277057 +AAAAAAAAIAOBAAAA Rig Sports baseball 4.47 7838.81 2.67459928573946137 +AAAAAAAAIBNAAAAA Easily natural relatives used to walk thorough, real rocks. Front implications tell either. Members achieve in a words. So black ages help far Sports baseball 90.17 13337.28 4.55067536548368992 +AAAAAAAAIFAAAAAA Teachers might not send unusual arrangements. Complex steps ought to hold all but statistical, recent pr Sports baseball 7.75 1162.44 0.39662412964658915 +AAAAAAAAIGCEAAAA Kids live so other goods. Colleagues ought to gain at first burning guidelines. Electronic, public figures give. Little leaves interfere. Stages could not determine yet environm Sports baseball 3.90 6580.60 2.24529846491203378 +AAAAAAAAIHACAAAA Only solid days cannot cope ever suitable recordings. Inches go ever chro Sports baseball 9.36 11126.11 3.79622491922354013 +AAAAAAAAJCEBAAAA Cities ought to assess to the parties. Likely organs help domestic, passive stages. Therefore private obligati Sports baseball 1.03 7447.72 2.54115951176103277 +AAAAAAAAJFKBAAAA Hundreds would give seldom national, philosophical words. Obvious things li Sports baseball 2.21 83.50 0.02849017138561147 +AAAAAAAAKDOCAAAA Most local companies shall see already. Politicia Sports baseball 18.00 3997.41 1.36391492213840880 +AAAAAAAAKEJAAAAA Surprising applications could not explore. Tonight expensive layers meet then between a statements. Days de Sports baseball 0.95 4521.40 1.54270013057369686 +AAAAAAAAKJDDAAAA Offices obtain surprisingly according to the cups. Separate, only children work also social rates. Public conflicts force at least. Gradually australian storie Sports baseball 1.45 8302.97 2.83297051867772986 +AAAAAAAAKKOAAAAA Conscious, solar ambitions support outside countries; warm facilities rise occupations. Appropriate columns grow. Availabl Sports baseball 3.35 2187.71 0.74644590229959357 +AAAAAAAAKLHAAAAA Certain places kn Sports baseball 4.63 546.48 0.18645878872825095 +AAAAAAAAKMDAAAAA Single, wonderful departments will appea Sports baseball 3.19 5797.68 1.97816642920876516 +AAAAAAAAKNMAAAAA Statutory Sports baseball 4.72 3059.64 1.04394811950026670 +AAAAAAAAKNPCAAAA No scottish accidents will rely chan Sports baseball 4.35 25561.00 8.72140444057023607 +AAAAAAAALKHAAAAA Properly common methods remember thus successful levels. Statistical families exist; trees will not go simply. Bottom, full things could see in the feet. Used, de Sports baseball 2.57 12848.83 4.38401639286929566 +AAAAAAAALOGAAAAA Good effe Sports baseball 9.77 8394.54 2.86421417129785492 +AAAAAAAALPIAAAAA Central standards ease written eyes. Simple, brief groups send in the ideas. Technical, possible islands try on a parties; activities must change adul Sports baseball 5.06 9693.92 3.30756218201684687 +AAAAAAAAMBFEAAAA Legal, other houses compete well new companies. Young, able layers would find orders. Rather good beaches die finally historical applications. Comments Sports baseball 89.48 2008.38 0.68525856775370489 +AAAAAAAAMHBAAAAA Clubs may take major changes. Procedures need. Lawyers shall not say pretty Sports baseball 1.61 8727.74 2.97790189711445061 +AAAAAAAAMKECAAAA Clear practices might not own as. External Sports baseball 1.32 525.24 0.17921170800692895 +AAAAAAAAMLNBAAAA As simple views visit only japanese, direct differences. Hours assist locally too severe products. Else lesser dangers telephone Sports baseball 7.20 316.92 0.10813299539554474 +AAAAAAAANBEBAAAA Anxious, just years must come various schools; rarely surprising students ought to talk complex hundreds. Thin, other makers shall look actually american, ta Sports baseball 7.88 11407.21 3.89213614289414352 +AAAAAAAANCKAAAAA Too particular pages used to give here by a markets; capital, different researchers gain specialist, small directors. Required patie Sports baseball 60.56 503.66 0.17184861940212062 +AAAAAAAANEODAAAA New friends would leave long motives. Dogs shall face occasionally increased schools. New, green parents decide also probably beautiful men. Real tanks shall Sports baseball 0.54 928.53 0.31681411780457264 +AAAAAAAANNKBAAAA Important, private results identify sh Sports baseball 1.25 4287.60 1.46292765069398475 +AAAAAAAAOCNAAAAA Other, significant materials could not mention economic, current races. Animals go straight living, young groups; masters may date. Top, able computers avoid less hours; questions recommend Sports baseball 0.56 225.54 0.07695417071030911 +AAAAAAAAOLFCAAAA Only warm clouds ought to hold really Sports baseball 4.99 1216.60 0.41510350308664564 +AAAAAAAAOOEAAAAA Books change slightly. Radical, distinguished characteristics imagine always as a ministers. Red strings deal late, sexual states. Peculiar, strong patterns live always. N Sports baseball 1.51 2123.42 0.72451017633095930 +AAAAAAAAOONAAAAA Real, social cigarettes wou Sports baseball 0.29 5316.32 1.81392656216471802 +AAAAAAAAPLODAAAA At least middle departments arrange international, environmental sites. More key kids might take up to the relations. Policie Sports baseball 4.87 2378.20 0.81144102502109211 +AAAAAAAAABKDAAAA Young workers ac Sports basketball 7.78 1526.51 0.57071382054190651 +AAAAAAAAADCAAAAA Inter Sports basketball 85.58 1184.67 0.44291065357015702 +AAAAAAAAAIADAAAA Levels evaluate old arms. Attractive, dangerous men isolate very poor things; solid, sorry others shall leave now Sports basketball 1.44 153.89 0.05753460497683867 +AAAAAAAABEJCAAAA Others ought to ensure still buildings; new patients keep notably in a drivers. Relative, good im Sports basketball 1.20 625.50 0.23385467160317491 +AAAAAAAABHCAAAAA Favorite, pure features see green decisions. Imp Sports basketball 8.03 5094.18 1.90455282332128144 +AAAAAAAABIHDAAAA Also federal cells shou Sports basketball 6.62 8298.39 3.10250562475630792 +AAAAAAAACAMCAAAA Considerable ears cross during a members; very southern politicians allow numbers. Patients deprive earlier shares. Men used to press beautiful tactics. Eyes might develop on a co Sports basketball 4.97 937.69 0.35057264111204009 +AAAAAAAACBKDAAAA Youn Sports basketball 3.28 1166.47 0.43610624905668334 +AAAAAAAACECDAAAA Always front rumours ought to improve. Hours use about a centuries. As uncomfortable links learn neither about real reasons. Dark days could deal much big, sole Sports basketball 6.68 10473.18 3.91559083859462726 +AAAAAAAACIFEAAAA About national assets raise much. Other inhabitants may like thick annual characteri Sports basketball 6.72 1181.36 0.44167314923281648 +AAAAAAAACKABAAAA Early types tell links. Local reasons succeed probably properties. Friends carry low fruits. Able, old tensions get. Recently other vegetables Sports basketball 3.00 11903.67 4.45040581730226223 +AAAAAAAACKKDAAAA Cases should soften courses; complex letters use experimentally far whole parties. Great, liberal decisions confirm. Households know very reasonable users. New, short feature Sports basketball 2.58 5361.15 2.00436446469282357 +AAAAAAAACMJAAAAA At all attractive answers would beat. Trousers might take of course fine constant lives. Ladies shall not challen Sports basketball 8.87 19675.51 7.35605104664266008 +AAAAAAAACPGAAAAA Whole councils would see again white Sports basketball 4.23 4485.02 1.67680716104503839 +AAAAAAAADGNAAAAA So early systems would place only to a m Sports basketball 2.69 249.12 0.09313809079101988 +AAAAAAAAEILCAAAA Different plans may make so in a trials. Provincial, perfect items must wear together. Simple aspects must not prefer then in the sections; alone, good rights can put psycho Sports basketball 4.46 9055.60 3.38560250067100027 +AAAAAAAAENEBAAAA S Sports basketball 1.06 458.00 0.17123171797642543 +AAAAAAAAFEDEAAAA Often final groups participate with the characters. Superior, in Sports basketball 62.36 9883.09 3.69497484632233713 +AAAAAAAAFKMBAAAA Decisions bring young farmers; easy other minerals credit preliminary, basic offices. Sports basketball 0.22 9644.13 3.60563525827070695 +AAAAAAAAFMCDAAAA Properly large others say briefly other police. Results used to prefer worried, old opportunities. Very big contents create forces. Possible, famous clu Sports basketball 4.35 9926.05 3.71103623192117389 +AAAAAAAAHPIBAAAA Succ Sports basketball 9.92 8445.05 3.15733716134675017 +AAAAAAAAIBBBAAAA Similar, new events may need sometimes combined prisons. Communications pay from a relat Sports basketball 20.67 3976.01 1.48650441701189361 +AAAAAAAAIBFCAAAA Charming, general guns would look superficially; big heads can set essentially straight voluntary refuge Sports basketball 0.21 5246.26 1.96141072653057135 +AAAAAAAAICGEAAAA Authorities might destroy however to the profits. S Sports basketball 2.28 2179.53 0.81485734995886139 +AAAAAAAAINLDAAAA Favourably major feelings used to turn new, necessary years. Labour products go pr Sports basketball 7.28 256.36 0.09584489786121490 +AAAAAAAAIOKDAAAA Different organizations shall split; emotional, com Sports basketball 2.22 12749.88 4.76677697902460058 +AAAAAAAAJGLCAAAA Smooth years help more british, curious arms. Inter alia acute members must improve also in a years. Now regional Sports basketball 3.91 2159.38 0.80732390210465840 +AAAAAAAAJNMAAAAA Women may not represent very common muscles. More late stones smile again on the surveys. Topics must not find as variations. Economic boots Sports basketball 60.56 202.95 0.07587658769282869 +AAAAAAAAKACBAAAA Heavy paintings s Sports basketball 4.08 4622.30 1.72813181223238268 +AAAAAAAAKADAAAAA Huge, helpful heads think low policies. Absolute tons restore generally. Tradit Sports basketball 5.01 24011.93 8.97730136644032550 +AAAAAAAAKDDDAAAA White interests might Sports basketball 53.99 3630.36 1.35727681151287298 +AAAAAAAAKOJCAAAA Outstanding friends must reduce figures. Travellers Sports basketball 0.95 3994.52 1.49342472072312426 +AAAAAAAALAEEAAAA Redundant, new writers draw sharp Sports basketball 4.80 9195.80 3.43801884752753924 +AAAAAAAALJEEAAAA Clear members work national, personal operations. He Sports basketball 4.17 4072.64 1.52263131855788049 +AAAAAAAALKKAAAAA Times remove other effects. Almost english conservatives can measure however new, normal r Sports basketball 7.65 1107.60 0.41409661753425504 +AAAAAAAAMHCCAAAA Now due eyes keep about. Then annual progr Sports basketball 0.83 3016.20 1.12766180733732398 +AAAAAAAAMHPBAAAA Addresses retain once more applicable events. Following blocks follow for a develo Sports basketball 70.89 268.59 0.10041730814691726 +AAAAAAAAMKFCAAAA Other, g Sports basketball 0.70 15012.84 5.61282616791528113 +AAAAAAAAMNFBAAAA Political aspects ought to say months. Of course Sports basketball 3.77 123.24 0.04607553913409317 +AAAAAAAANNHBAAAA Fortunately favorite decisio Sports basketball 2.86 9079.28 3.39445570390611328 +AAAAAAAANOEDAAAA Ancient, similar ways equip immediately. Never european leader Sports basketball 0.67 5371.94 2.00839850451152582 +AAAAAAAAOCJAAAAA No doubt established kinds ensure both comparative buildings. Threats attract almost traditional students; questions must not fight widely clean, minor relations. National, famous assets go commer Sports basketball 9.10 1401.61 0.52401765989724377 +AAAAAAAAODJCAAAA Only social changes could achieve again soon go Sports basketball 9.05 4303.38 1.60889770852705168 +AAAAAAAAOEFCAAAA Early favorite contexts will save quite as empty pages. Unusual languages suffer soon actual cars; corporate businesses ought Sports basketball 54.80 7564.49 2.82812362077617992 +AAAAAAAAOHGCAAAA Recently free woods know Sports basketball 2.84 3637.05 1.35977799097414435 +AAAAAAAAOJLBAAAA Confidential members cannot modify either dirty organisations. Men might think increasingly failures. Internationa Sports basketball 1.70 6383.10 2.38643925549196761 +AAAAAAAAOLMDAAAA Old, poor pp. form seconds; bags know much; Sports basketball 9.50 5416.98 2.02523753634047386 +AAAAAAAAPLEBAAAA Comparatively unable miles show already; interesting drugs will not run parts. Yet political priests will run strangely left, d Sports basketball 4.52 1863.76 0.69680093165009314 +AAAAAAAAPNOBAAAA However comprehensive times ought to level even for a blacks. New employers see; far presidenti Sports basketball 4.48 4373.10 1.63496381197097391 +AAAAAAAAPPGBAAAA Areas expect. Organic, democratic resources would last previously. Cheap, residential fields cannot achieve seriously about Sports basketball 0.77 2524.50 0.94383072495957642 +AAAAAAAAACAAAAAA Automatically competitive deaths jump wooden friends. Average, legal events know. Losses ought to cross. Conventional toys st Sports camping 4.38 8168.10 2.37504813353538829 +AAAAAAAAANABAAAA Only far tests take to a others. Appropriate comparisons will say fully musical personnel. Beautiful, administrative aspects get standards. Huge, sin Sports camping 1.74 11263.88 3.27521175920551774 +AAAAAAAABCABAAAA Cells give only serious walls; arrangemen Sports camping 0.18 151.45 0.04403729628970441 +AAAAAAAABGCBAAAA Sorry eyes could shake obviously across a commentators; more other numbers may control schools. Children maintain. Powerful elements gather very then active opportun Sports camping 3.69 5210.19 1.51497313143383954 +AAAAAAAABHCDAAAA A bit important Sports camping 3.97 2060.79 0.59921835471020101 +AAAAAAAABHEEAAAA Straightforward deal Sports camping 4.48 14808.62 4.30592001704617007 +AAAAAAAACCNCAAAA Whole services live since the wheels. Sports camping 2.26 8417.24 2.44749086709509087 +AAAAAAAACFDEAAAA So-called, classical travellers contain capital, new paintings. Japanese stories Sports camping 6.17 18270.48 5.31252915889810863 +AAAAAAAACIABAAAA Financial, massive ideas might boil also leading companies. Even long Sports camping 9.92 4367.79 1.27002748340183563 +AAAAAAAACLOCAAAA Groups should display of course possibly productive areas. Gro Sports camping 2.04 12234.96 3.55757384359644646 +AAAAAAAACOJAAAAA However general jobs tell basic results. Issues lose critical husbands. Back, Sports camping 21.20 4822.68 1.40229638871199501 +AAAAAAAADBIBAAAA Equal, different facts emphasise necessary inhabitants. Complex, active moves might put in a reports. Commercial groups can restrict curiously to a players; identical purposes cou Sports camping 8.94 13999.26 4.07058144903669396 +AAAAAAAADCLDAAAA Always opposite skills take well in the prices. Colonial, weak issues shall deny more just respective funds; mental, creative patients would not play even in Sports camping 16.73 5674.31 1.64992585480113970 +AAAAAAAADJFCAAAA Procedures find groups. Possible Sports camping 4.18 5862.76 1.70472168501437704 +AAAAAAAADKIBAAAA Wild changes shall delay soon representatives; other countries die also legal, superb boys. Never video-taped sounds think substantially previous approa Sports camping 75.50 1678.45 0.48804489902577986 +AAAAAAAAEADDAAAA Dear officers communicate much long interested relationships. Casualties position normal, good issues. Aspirations remind now quick words. Financial, l Sports camping 3.38 1526.49 0.44385930943064297 +AAAAAAAAEGBCAAAA Exceptions say richly normal losses; british, old individuals used to win. Childr Sports camping 4.27 4862.61 1.41390688221379690 +AAAAAAAAEGCEAAAA Then bad merch Sports camping 0.84 409.38 0.11903590858421386 +AAAAAAAAEGKAAAAA More fine pressures free there into the records; rights turn seconds; great areas ought to drain allegedly especially gothic dealers; programs speak even european, o Sports camping 2.25 4430.31 1.28820649802073507 +AAAAAAAAEJGAAAAA National systems must believe old issues. Long police would make able towns. Slow years earn exactly nearer the terms. Social, old comparisons shall survive wildly previous children Sports camping 2.12 4781.18 1.39022938444641077 +AAAAAAAAEMKCAAAA Well main goods share probably traditional times. Enorm Sports camping 5.17 3862.11 1.12299030949772389 +AAAAAAAAEOKAAAAA Terms reduce standards. Free things put Sports camping 2.60 1759.84 0.51171076594568109 +AAAAAAAAEPAAAAAA Players must argue away significantly national sides. Elections might Sports camping 3.53 14678.84 4.26818373238141050 +AAAAAAAAFDBBAAAA Labour, bright taxes could not shock still with a reasons. Dominant weapons will cause home; women say therefore bloody, complete areas; dem Sports camping 30.04 3575.90 1.03976868803138980 +AAAAAAAAFJODAAAA Unable school Sports camping 2.63 9178.29 2.66878227905467845 +AAAAAAAAFLMDAAAA Still royal companies reach years. Complex, british plants must tell however such as a detectives. Ite Sports camping 6.35 8374.50 2.43506330655747472 +AAAAAAAAFNHDAAAA Just capitalist exceptions communicate Sports camping 7.91 397.64 0.11562225484739558 +AAAAAAAAFPNBAAAA Available tests forgive also policies. Almost local rights used to argue there new only men. Chi Sports camping 2.78 316.16 0.09193021852065332 +AAAAAAAAGNGDAAAA Never top observations spend appropriate, common states. Homes make. There available hospitals will appreciate away upon a years. Roots hang Sports camping 2.07 4784.91 1.39131396097437774 +AAAAAAAAHEBBAAAA Residents will l Sports camping 7.50 7103.96 2.06562688247083863 +AAAAAAAAHGIBAAAA Bold campaigns get with a numbers. Public, medical emotions recognize sources. Very single countries shall fit enough along with Sports camping 4.72 5615.05 1.63269475425225965 +AAAAAAAAHGJDAAAA Democrats may say again. There private services can think about fa Sports camping 1.65 18235.67 5.30240741387437400 +AAAAAAAAHKNBAAAA Different, ltd. students may not try scottish sheets. Almost likely schools may not order. Partly effective c Sports camping 3.91 11958.94 3.47731518052689077 +AAAAAAAAIIHCAAAA Certain, official generations might allow polish letters. Months provide equally product Sports camping 8.26 3715.04 1.08022659100761608 +AAAAAAAAIKDCAAAA Central, clear fingers must Sports camping 5.58 104.95 0.03051643608850761 +AAAAAAAAILADAAAA Always clinical doors Sports camping 33.45 2954.82 0.85917651913334019 +AAAAAAAAJMICAAAA Available implications try only; magistrates must reduce quite black, ugly girls. Animals read. Chief pupils will manipulate easy more real seconds. Men might throw only british policies. Aspects ex Sports camping 6.42 12904.54 3.75226841506993789 +AAAAAAAAKCJBAAAA Affectionately sad chains answer sideways small, concerned documents. Interested minutes notice as a yards. Difficult c Sports camping 0.18 7683.32 2.23408807744213704 +AAAAAAAALALAAAAA Crucial sources make to a police. Great farmers make recent limitations. Yet indian colleagues should get. Mea Sports camping 7.95 1656.32 0.48161013265475868 +AAAAAAAALDKBAAAA Good, white statements de Sports camping 8.79 4572.10 1.32943494464283601 +AAAAAAAALHICAAAA Conventional workers shall not take numbers. French, premier things could remember as to a gardens. Red districts ought to implement flowers. Fiscal, curious terms study much explicit words. Third Sports camping 3.61 5559.40 1.61651333768889187 +AAAAAAAAMBGDAAAA Fresh, electoral doors get at a teachers; children become more ministers; comfortable places shall not lift much safe, genuine procedures; official, extra beliefs break. Openly new days find ther Sports camping 1.27 4702.53 1.36736023057922522 +AAAAAAAAMELBAAAA Much basic birds can light apparently. Normal, close teeth cannot expect as civil ends. Long principal conditions could not cover less more new officers. Efficient words get to a years. Real, able Sports camping 1.68 3665.26 1.06575200131265745 +AAAAAAAAMMOAAAAA Far specific clothes learn indeed times. Gastric, steady criteria imagine again in n Sports camping 50.85 6713.37 1.95205456449265676 +AAAAAAAANGMAAAAA Grounds will take then by the boards. Historical candidates feel suitable numbers. Normally inevitable savings return systems. Psychological rooms would turn almost Sports camping 2.39 16931.42 4.92316909306983803 +AAAAAAAANJJAAAAA Accounts listen firmly incredible trends. Votes must not exert away natural fears. Able terms reflect well golden others. British feet could not re Sports camping 8.64 12203.84 3.54852504425319390 +AAAAAAAAOBLCAAAA Labour patients shall Sports camping 2.75 7756.62 2.25540160545821715 +AAAAAAAAOHPBAAAA Powerful populations can produce honest lines; soviet, working-class feet w Sports camping 2.14 2940.02 0.85487310556392702 +AAAAAAAAOKHDAAAA Minutes can compete much mathematical areas; pages put for example more good passengers. Differences undertake to a parts. About conscious situations know light studies; mad, l Sports camping 1.46 2184.90 0.63530596674397594 +AAAAAAAAOLDEAAAA Visual, surprising parties earn resources. Particular, just situations can lose currently to a others. Social actors want loudly prime years. Fresh, other responsibilities obtain offices. Afraid t Sports camping 9.02 6215.95 1.80741916059417696 +AAAAAAAAONBDAAAA Great explanations would not fill; sure, political powers let eventually horses. Continually public examples ask yet wrong, dependent officials. Early, g Sports camping 1.82 3966.35 1.15330029804337451 +AAAAAAAAONHBAAAA Trustees could respond further precise surveys. Conditions would weigh. White areas secure particularly living costs. Strong, bare provisions can keep so useful, physical feet. Demanding, supreme Sports camping 4.48 9027.65 2.62498050742654327 +AAAAAAAAPBAAAAAA Just available years undertake social units. Alone long-term years communicate very huge workers. Relevant, false farmers start hardly bottom windows. Associations shall Sports camping 7.57 5611.89 1.63177591730095251 +AAAAAAAAABGAAAAA Steps would make repeatedly short pairs. As good stages protect skills. Plants could not sweep observations. C Sports fishing 8.71 4424.59 1.05964726402462346 +AAAAAAAAACDDAAAA Christ Sports fishing 9.05 1582.84 0.37907514038334286 +AAAAAAAAAFIBAAAA Almost personal matters may deal; major, australian offences happen prime, usual hours. Functions might visit at the followers. Championships shall smile observations; compani Sports fishing 2.61 1554.46 0.37227840004061759 +AAAAAAAAAGICAAAA Accidentally wrong communities look more goods. Rural matters recognize. Large, new days go hap Sports fishing 1.32 4303.95 1.03075513030558269 +AAAAAAAAAGMCAAAA Problems ought to remove rapidly then new authorities. Half way exotic months bar systems. Front, new models cause too; difficult, full others comprehend eve Sports fishing 2.89 2105.84 0.50432867101214193 +AAAAAAAAAJPBAAAA Delightful, married guns should go much tremendous, clear networks. Again just hours shall know there. Large, whole years cannot want Sports fishing 9.33 2187.51 0.52388786001109799 +AAAAAAAAALDAAAAA Very modern weeks must prevent hotly to a situations. Points look strongly regulations. Times take good groups. As af Sports fishing 68.83 2026.90 0.48542329107363830 +AAAAAAAABEFDAAAA Members support general, mysterious programmes. Front times want with the services. Now new details should impose never cheap live activiti Sports fishing 4.96 11202.69 2.68293781078382606 +AAAAAAAABHGCAAAA Tests shall see famous, good words; sexual, significant theo Sports fishing 8.63 11684.99 2.79844407813042221 +AAAAAAAACBEEAAAA Personal, lacking artists cut pieces. Prices make quickly for a rooms. High, overall types ought to use together supposed women; reductions shall give prices. Lengthy, fundamental meas Sports fishing 9.23 13101.80 3.13775661107533389 +AAAAAAAACEMBAAAA Other offices shall embark blindly resources. Spectacular copies may look also old, other offices. Properties fill better important others. Very wrong supplies will not own both aspects. Certainly Sports fishing 7.25 386.95 0.09267084833042791 +AAAAAAAACKDBAAAA Sheets identify in a persons. Successful studies cannot solve for instance impressive governments; public buildings can move to a women. Substances sweep even on a tales; however great spac Sports fishing 4.50 5339.33 1.27871880247087137 +AAAAAAAACLCEAAAA Inherent, public girls run. Opposite, similar players might adjust though central ties. Entirely small generations achieve rats. At all western boxes prosecute almost suspicious, ordinary v Sports fishing 0.46 2861.92 0.68540264699268189 +AAAAAAAACPJAAAAA Difficult skills can tell specifically significant applicants. Irish women find si Sports fishing 8.65 0.00 0.00000000000000000 +AAAAAAAADCPCAAAA Usually english commentators will indicate still dangerous, spiritu Sports fishing 9.90 13087.32 3.13428878865945433 +AAAAAAAADDDBAAAA Early, associated parents continue stories. Alive, key costs will not supply. For example excellent wi Sports fishing 0.65 9375.15 2.24525934545809862 +AAAAAAAADEDDAAAA Just left grounds would not shoot other, accessible readers. Long, true winners shall vary; male conditions must hear never local, clean studies. Major, generous pp. must not get always gre Sports fishing 3.62 8.19 0.00196142718135729 +AAAAAAAADFMCAAAA Groups deserve also only members. Inevitable, rare dreams worry much old enquiries. Please clear nerves turn necessar Sports fishing 2.58 3603.80 0.86307585789687587 +AAAAAAAADFPCAAAA Foreign advances expand never new, colonial players. Colours confess lines. Urgent, massive items sit then men. Different countries cut however. Effectively old ideas suggest only actually particul Sports fishing 4.19 20.28 0.00485686730621806 +AAAAAAAADGICAAAA Sole, public skills require long opportunities. Parents destroy how Sports fishing 4.84 1396.88 0.33453948731311060 +AAAAAAAADKFBAAAA Courses try military parents. Fast, w Sports fishing 1.64 6454.18 1.54571478453878082 +AAAAAAAADLIAAAAA New parties strengthen please at all current things. Similar teams must lead most real firms. Simply tiny planes will set moving advantages. Concerned, average memories use Sports fishing 2.13 5552.34 1.32973267352104439 +AAAAAAAADOHBAAAA International, new heads succeed altogether. Inc men see about accord Sports fishing 4.11 4917.54 1.17770410517847910 +AAAAAAAAEABAAAAA Illegal campaigns ought to become here western, certain abilities. Indirect teachers would not tend no longer long, main agreements. Twice sweet patients ought to enjoy Sports fishing 0.33 2469.18 0.59134514867689882 +AAAAAAAAEAOBAAAA Common, preliminary children will not maintain early further international Sports fishing 3.67 4265.38 1.02151798178483168 +AAAAAAAAEBGCAAAA Northern, de Sports fishing 15.22 1489.04 0.35661093163959266 +AAAAAAAAEJHDAAAA Unable occasions command more effective, other birds. Proper songs know in a ports. Later wealthy details look now hours. Aware, black issues Sports fishing 0.59 4257.58 1.01964995589782473 +AAAAAAAAFEICAAAA Points can appoint even pregnant ideas. Other, basic bodies shall frighten too modern laws; features accompa Sports fishing 1.97 15202.78 3.64092135826557149 +AAAAAAAAFFMAAAAA Home available features need with a questions. Hard waters can operate still more content bands. Organic, large ideas contribute points. Difficult, right produc Sports fishing 2.47 7589.73 1.81766821992220870 +AAAAAAAAFJCAAAAA Collective, full signals will assume only services. Political villages think children. So old Sports fishing 2.56 2552.33 0.61125878361338953 +AAAAAAAAGDABAAAA Industrial, slight needs would disturb too for a folk. Now known buildings ought to suggest so. Papers create colours. Good levels tell into a r Sports fishing 2.72 5261.10 1.25998346078618504 +AAAAAAAAGNJAAAAA Norma Sports fishing 1.01 8662.39 2.07456009786539724 +AAAAAAAAGOCBAAAA Onwards horizontal sports find. Normal, powerful eyes come american, commercial situations. Major, enormo Sports fishing 1.89 13071.78 3.13056710631534049 +AAAAAAAAHACEAAAA Shoes give more now annual ch Sports fishing 1.18 6235.99 1.49346035270723652 +AAAAAAAAHIJBAAAA As modern women may find only with a bones. However simple minutes end changes. Catholic hands provide hard able rights. Weeks used to affect then tiny c Sports fishing 2.55 3728.50 0.89294032303915358 +AAAAAAAAHODBAAAA Strong, southern weeks use to a exceptions. Shoulders write natural, particular courses. Cold, labour things will hang. New authorities may bu Sports fishing 1.08 5888.16 1.41015837267164344 +AAAAAAAAHPCBAAAA Automatically private stands go always easy terms. Well distinctive depar Sports fishing 1.17 5365.88 1.28507727520164501 +AAAAAAAAIBBDAAAA Internatio Sports fishing 1.86 8437.51 2.02070347459999698 +AAAAAAAAIBIAAAAA Apparent, Sports fishing 7.13 2649.10 0.63443427913719237 +AAAAAAAAICNDAAAA Special, easy things invest here hot Sports fishing 4.61 8905.67 2.13282334630014721 +AAAAAAAAIDMDAAAA Leaves could not help accounts; maximum, supreme expenses may not build in a officers; r Sports fishing 0.44 13341.40 3.19513853447621392 +AAAAAAAAIGFAAAAA Still original workers solve merely villages. Only long years punish already. Scottish features should not take from th Sports fishing 4.81 3.50 0.00083821674416978 +AAAAAAAAJFODAAAA Settlements must make significa Sports fishing 7.42 7154.29 1.71338447732755427 +AAAAAAAAJHDDAAAA Shortly new terms would recover yet satisfactory, previou Sports fishing 2.86 3393.96 0.81282117172642234 +AAAAAAAAJHKCAAAA Public, certain lives could not choose indeed in a tools. Then bad things gain women. Sports fishing 2.62 392.55 0.09401199512109957 +AAAAAAAAJJEAAAAA Circumstances cannot take lines. Modern goods would make corresponding tools. Subsequently toxic practices see annually alm Sports fishing 3.56 12846.92 3.07671527285990692 +AAAAAAAAKGAEAAAA Also normal groups must not keep possibly others. Rates will not depend centuries. Fields could indicate already in a months; important arti Sports fishing 64.57 16106.48 3.85734892161020958 +AAAAAAAAKIACAAAA Crops shall argue already for the responses. Easy committees like as with a figures. Easy current groups should not meet nevertheless; evident, international forces sen Sports fishing 6.00 1274.25 0.30517076750238473 +AAAAAAAAKILCAAAA Elements take further vital, typical Sports fishing 1.73 6796.42 1.62767801268868558 +AAAAAAAAKIMBAAAA Good, silent examples close so literary, financial years. Often foreign interests discourage best suddenly whi Sports fishing 4.19 4776.06 1.14382098947415311 +AAAAAAAAKKBCAAAA Projects support indeed in a departments. Populations involve even with a terms; fine, classical miles visit continuously crucial, great days. Steady, sc Sports fishing 0.68 7528.93 1.80310719762348789 +AAAAAAAAKOACAAAA Directions use none the less. Military, new recordings pass yellow tasks. Frequently wor Sports fishing 1.49 1880.44 0.45034751268760788 +AAAAAAAAKOIDAAAA Poor networks explain personally to a funds. Already federal words accelerate companie Sports fishing 2.01 7024.79 1.68237045779327228 +AAAAAAAALBICAAAA Sectors might not know properly. Large, electric workers used to drop even as ca Sports fishing 6.89 1774.46 0.42496630967414683 +AAAAAAAALOIBAAAA Old others will cut other activities. Sharp passages avoid allegedly orthodox, additional firms. High officers must form. Sports fishing 0.25 2612.13 0.62558031541377612 +AAAAAAAAMAEBAAAA Very acids should depend much a little christian tons. Warm rules defeat at large details. Banks should not seek then. Times can back stiffly ordinary, chemical Sports fishing 6.07 10778.84 2.58142976306486528 +AAAAAAAAMEMDAAAA Factors might assist now absent, voluntary demands; political companies might know no longer concerned things; autonomous, possible events can dry at Sports fishing 6.68 6637.53 1.58962536740836076 +AAAAAAAAMIIDAAAA Pale, other animals assist in a words. Is Sports fishing 3.40 1226.34 0.29369677772719206 +AAAAAAAAMKLCAAAA Necessary women know little international troops. Immediate, possible drugs can try effectively too gentle spots. Northern, german ideas tell to a areas. False appropriat Sports fishing 2.18 505.79 0.12113189915246708 +AAAAAAAAMNAAAAAA Western, social things will go in order. Warm, male cards used to describe. High, briti Sports fishing 0.51 2346.30 0.56191655624158939 +AAAAAAAAMNEDAAAA Different, common buildings could not comply even. Impossible transactions build always qualities. Police move tiles. Options must use just different stages; words Sports fishing 8.87 4167.10 0.99798085560854416 +AAAAAAAANCGBAAAA Members like very. Then interested principles could remember yet important, new agents. Necessarily due assets generate across. Areas give anyway as social projects. Main, Sports fishing 1.79 7991.56 1.91390268686784986 +AAAAAAAANECDAAAA Blind sessions hurt traditionally public, various clothes. High, southern schools might not tal Sports fishing 1.43 1122.60 0.26885203342999968 +AAAAAAAANMFBAAAA Practical roads mean either dishes. Necessary issues determine simply fund Sports fishing 3.40 4810.52 1.15207383204675046 +AAAAAAAAOCHDAAAA Just formal teams ask still, certain interests. Well l Sports fishing 9.79 2218.77 0.53137433298902583 +AAAAAAAAOEKDAAAA Books must work major, able forces. Clearly future teachers would measure certain, direct measures. Hard tears go main nurses. Cruel patients used to leave much only days. Yet social defence Sports fishing 8.56 1810.80 0.43366939438361253 +AAAAAAAAOGNCAAAA Comprehensive, able acts must not resign. British, red forces convict perhaps; years want as well problems Sports fishing 54.91 119.66 0.02865743303067322 +AAAAAAAAOIOCAAAA New companies must benefit always to a companies; adults might get yet international, comfortable indicators. Dual bones shall find ever like parents. Wars need new, heavy purposes Sports fishing 3.43 7734.29 1.85228896636140409 +AAAAAAAAOLKDAAAA Backs think now back, british wines. Very fine shows get often serious, fatal prisoners. Good terms ought to come far easy, obvious shoulders. Machines play more ac Sports fishing 2.94 7583.99 1.81629354446177025 +AAAAAAAAOOFEAAAA Tiny values allow equations. Sports fishing 4.39 7729.84 1.85122323364381680 +AAAAAAAAPBKCAAAA Ill, simple objects shall bear solid trees. Ears should use there minimum, inappropriate personnel. Available practices should not apply increasingly pr Sports fishing 7.87 15557.69 3.72591893102937088 +AAAAAAAAPDKAAAAA Sure reliable suppliers show upright other women. Maybe Sports fishing 1.11 12392.70 2.96793389870653577 +AAAAAAAAPEFEAAAA Much common times achieve existing, continuing positions. Los Sports fishing 8.20 9965.46 2.38663298152977430 +AAAAAAAAPHKCAAAA Good, whole facilities maintain for a points. More worthwhile directors battle annual hours. Yes Sports fishing 8.90 603.00 0.14441277049553697 +AAAAAAAAPLGCAAAA Rules offer. Important, italian goo Sports fishing 4.06 3150.39 0.75448847104715544 +AAAAAAAAPLIAAAAA Vital, similar activities change thickly. Seats would sit essentially brilliant words. Hig Sports fishing 68.38 6302.32 1.50934575746174558 +AAAAAAAAPMMBAAAA Even useless times make old, old studies. Early public employees must open together warm consequences. Sufficient, evident men would operate stars. Various, other sections control l Sports fishing 89.62 2679.48 0.64171000047658609 +AAAAAAAAAFKCAAAA A Sports fitness 7.12 10468.61 4.22441966945963305 +AAAAAAAAAHMDAAAA Fast bizarre situations fulfil all as political plans. Thus labour conventions run more part-time experiments. Early considerable own Sports fitness 0.81 5713.17 2.30544721056249987 +AAAAAAAAAPNAAAAA Other, cultural differences might take. Musical branches take only new defences. Sports fitness 3.76 18567.33 7.49251276543379958 +AAAAAAAABDGDAAAA Increased machines claim Sports fitness 1.76 2327.22 0.93910786084875139 +AAAAAAAACAHCAAAA New parties survive Sports fitness 1.06 5055.94 2.04023384036732070 +AAAAAAAACEFDAAAA Abruptly real years cope together; significant accounts provide at a others. Twice competent languages cannot impose most protests. Identical leaders Sports fitness 3.76 11311.78 4.56466578930728034 +AAAAAAAACGPCAAAA Clinical, real figures figure effects. Full, pleased bacteria used to fit immediately more main things. Windows will not present perhaps Sports fitness 4.25 1715.39 0.69221484579083182 +AAAAAAAACMADAAAA Concerned clothes comment less small probl Sports fitness 0.73 1855.00 0.74855195549816254 +AAAAAAAACOKCAAAA Large, working matters oppose etc far remote aspects; today amer Sports fitness 3.52 11563.15 4.66610164108818229 +AAAAAAAADKMDAAAA Physical questions confirm much to the marks. Irish, pleased eyes would know in an subsi Sports fitness 2.86 8639.15 3.48617392255630775 +AAAAAAAADOCAAAAA Little, national services will buy young molecules. In part video-taped activities join now Sports fitness 5.91 408.38 0.16479441918401058 +AAAAAAAAEBBEAAAA Intelligent trends used to bother open. Bedrooms will not hit all senior, economic boys; objects would sum. Often blue times should deal in a Sports fitness 3.84 1925.10 0.77683955230701493 +AAAAAAAAEBMDAAAA Absolutely wild standards impose only so scottish schools. New, complex incomes can establish children. Certainly free groups will rest. Impressive teeth must go front s Sports fitness 4.00 2927.91 1.18150552885316716 +AAAAAAAAEHJDAAAA Policies think races. Loc Sports fitness 40.32 1793.89 0.72389211183212873 +AAAAAAAAEIOAAAAA Shares could release barely months. Aware writings used to use so very impossible authorities. Sports fitness 6.66 3449.47 1.39197170562385268 +AAAAAAAAEMMBAAAA Boys might not fill in a problems. Military, young ways will encourage somehow inner, large matters. Ways will begin today whole firm Sports fitness 3.62 2731.00 1.10204603259594711 +AAAAAAAAFFPDAAAA Corporate heroes examine forth technical, formal shares; buildings may not emphasize abo Sports fitness 68.11 4428.60 1.78708204319092324 +AAAAAAAAFHBDAAAA Below old resources could cover lo Sports fitness 2.86 2908.84 1.17381017263141516 +AAAAAAAAFKKAAAAA Running children may continue common, small wives; great, subtle teams shall change bad, good lines; others may want; parties used to like near a sty Sports fitness 2.32 2591.76 1.04585822974766455 +AAAAAAAAFLCBAAAA Labour, dominant dreams come. Please various symptoms cannot persuade so owners. Primary colours would argue once small posts. Live, asia Sports fitness 48.03 4332.46 1.74828647176149287 +AAAAAAAAFMOAAAAA Deep, light measures could ask around experimental sections. Days attend social, wise cases. Children should find; as Sports fitness 3.91 12590.50 5.08067029417769025 +AAAAAAAAFNPBAAAA Times force also years. Emotional solutions ought to allow elderly differences. Too urban parents shall accommodate so. Traditional, effective im Sports fitness 3.60 8417.45 3.39671086674286159 +AAAAAAAAFOGAAAAA Principal eyes should pay frequently relevant areas. Light police m Sports fitness 3.17 451.78 0.18230771021830721 +AAAAAAAAGAABAAAA Original hands know as. So prime things might not identify. Less than little journals let very hard things; nurses see; large bodies name once political, national c Sports fitness 6.83 1540.63 0.62169358447392677 +AAAAAAAAGCMAAAAA Methods develop warm males. Governments depend please with the hospitals. At random tory weaknesses enter approximately simply young me Sports fitness 6.01 24.98 0.01008023064600760 +AAAAAAAAGENAAAAA Also new activities would not drop immediately fina Sports fitness 6.42 9171.55 3.70101438676505262 +AAAAAAAAGPBAAAAA Beings should affect close projects. In common labour metres might call directly Sports fitness 2.85 837.90 0.33811950593633983 +AAAAAAAAIAFBAAAA Men could not escape so old victims. Tiny horses give together effective teeth; little, beneficial bones used to forget again days. Of course Sports fitness 71.90 2421.19 0.97702776772646693 +AAAAAAAAIKEEAAAA Regions see in the cop Sports fitness 1.90 8595.06 3.46838219440648889 +AAAAAAAAINBDAAAA Asleep, fat topics pick into a rul Sports fitness 2.70 3452.62 1.39324283158601937 +AAAAAAAAJDJBAAAA Conscious, central results play above about the hands. Stages stay so available universities. Tomorrow professional birds decide; enthusiastically big views appear new window Sports fitness 9.62 412.47 0.16644486527456987 +AAAAAAAAJPAEAAAA Please positive sys Sports fitness 0.31 4494.44 1.81365059346046449 +AAAAAAAAKABCAAAA Simply necessary girls could not take supreme hospitals. Issues ought to Sports fitness 93.50 342.93 0.13838324641454710 +AAAAAAAAKAGEAAAA Overseas campaigns must finance just. Researchers believe sure, positive days. Workers appear from a values. Periods can lift ago related, extens Sports fitness 8.92 691.02 0.27884871821473869 +AAAAAAAAKDJAAAAA Regular, gold effects take gently for a terms. Good, strong difficulties attract articles. Ultimate farmers develop Sports fitness 1.12 3313.24 1.33699853425052940 +AAAAAAAAKFPCAAAA Round prisoners go at all into a lives. Streets find again places. Kindly liable men offer plainly on a contents. Early accurate regions should no Sports fitness 4.49 3281.89 1.32434780443658472 +AAAAAAAAKLAEAAAA More Sports fitness 0.82 1089.45 0.43962799348650845 +AAAAAAAAKNGAAAAA Solid, romantic feet would come so equations. Only economic feet will n Sports fitness 0.36 6592.06 2.66010749528906595 +AAAAAAAAKOICAAAA Only subjects think for a goods. Windows wo Sports fitness 3.66 9334.78 3.76688292352837611 +AAAAAAAALCIBAAAA Special miles must ease under across a conditions. Points might continue australian, australian places. Entirely Sports fitness 3.17 0.00 0.00000000000000000 +AAAAAAAALLDDAAAA Men mean also weapons. Individual proposals ought to mean farmers. Sometimes valuable eyes might take rights. Rough, different rewards cost real, alone ministers. Requirements may no Sports fitness 64.89 3913.00 1.57902091744706739 +AAAAAAAAMDKDAAAA Together working cases used to buy in a structures. Millions must Sports fitness 1.88 3472.20 1.40114398915402693 +AAAAAAAAMGABAAAA Sure, coming sessions could not pass very. Concerned children pick on a individuals. Easy pairs shall return. Reports consider subsequently rough sites. Vital, normal w Sports fitness 2.27 5967.84 2.40821471811329074 +AAAAAAAAMHMAAAAA Girls move ways. Other, human actors should participate serious families. New di Sports fitness 4.79 10717.00 4.32465299572712017 +AAAAAAAAMKDDAAAA Quick reasons could set only distant a Sports fitness 1.29 968.12 0.39066744968025936 +AAAAAAAANBGCAAAA So close miles would seem american, emotional horses. Other, alive operations ought to want further red layers. Parameters might faint bad, significant stations. So prime newspapers wou Sports fitness 2.97 9281.14 3.74523746428690903 +AAAAAAAANNEBAAAA Royal speeches take evil, front margins. For example hard events ought to go angles. Possible, foreign lakes shall not reconsider. Other honours hear momen Sports fitness 8.13 0.00 0.00000000000000000 +AAAAAAAAPDHAAAAA Points force into the symptoms. Local, strong negotiations get examples. For the time being fat result Sports fitness 5.61 19543.75 7.88652953114135530 +AAAAAAAAPFFDAAAA Subject, dead qualifications benefit more real nurses. Up to special writers give most responses; social circumstances de Sports fitness 2.69 12178.65 4.91447561877503891 +AAAAAAAAAIMAAAAA Just ready clothes try live skills. Girls investigate up Sports football 1.80 3028.92 1.26615656780156976 +AAAAAAAAAKKAAAAA Mostly furious applications cut in a workers; successful, substantia Sports football 3.20 4690.04 1.96054202463322710 +AAAAAAAAAMBDAAAA Dynamic, technical problems cannot go important, general sources. Overall inevitable subjects may take. Recent ends would n Sports football 2.51 10300.92 4.30601584472305176 +AAAAAAAAAOCEAAAA Allowances might lay at best children. Academic sections burst hot times. Short-term, warm goods Sports football 4.96 652.80 0.27288505720219244 +AAAAAAAABDPAAAAA Sophisticated, unfair questions may remove separate premises. Typical patterns intervene typically walls. Naked areas ought to return now military, necessary children; young met Sports football 33.19 7921.58 3.31139830182558766 +AAAAAAAACAGAAAAA Only available cars could not allow during a films. Cuts might not grow also unfortunately poor names. Windows go at first so key effects. Leading, possible relationships used to rec Sports football 1.80 5455.78 2.28063853765713464 +AAAAAAAACCDDAAAA Pupils talk tonight even expected rights. However federal costs may not borrow large decisions. Social, american soldiers repair legal, economi Sports football 11.06 1681.47 0.70289221374658476 +AAAAAAAACEBEAAAA British components must go. Wrong, overseas jobs explain with a towns. Quite ideological habits may Sports football 0.63 8173.32 3.41663127409899441 +AAAAAAAACGFAAAAA Girls would face criminal, special offenders. Healthy principles get very greek, ade Sports football 1.47 435.76 0.18215746404170861 +AAAAAAAACIECAAAA Delicate readers gain too able officers. Feet see as international appearances; just prominent samples halt just. Substantia Sports football 94.83 12471.06 5.21318309049015641 +AAAAAAAACNDCAAAA Daily, level areas fetch known, other Sports football 69.68 818.79 0.34227260414611390 +AAAAAAAACPNBAAAA More reasonable opp Sports football 3.70 3418.34 1.42894287137950754 +AAAAAAAADEKCAAAA Awful eyes get now like a gentlemen. Final countries may become french, turkish sciences. French lives repeat great, big standards. Large, able roads cl Sports football 6.18 5009.22 2.09396643112494858 +AAAAAAAADGHAAAAA Thanks may add suddenly strong weeks. Times abandon as files. Systems feel cheap targets. Green, formal events understand french, rea Sports football 0.97 2280.64 0.95335872680393409 +AAAAAAAADGKDAAAA Miserable officers introduce clearly. Much mathematical eyes could change so before prominent plans. Prices i Sports football 4.67 20055.07 8.38346955291662626 +AAAAAAAADLIDAAAA Else social offenders will not support mines. Gently intelligent expressions speed days. Sometimes old houses offer really important, local month Sports football 2.19 15388.53 6.43275105592471583 +AAAAAAAADPEAAAAA Critics can cover only str Sports football 1.79 10295.54 4.30376688392686948 +AAAAAAAAEBNBAAAA Sources negotiate never books. Sports football 12.71 1473.07 0.61577633457848288 +AAAAAAAAEHMDAAAA Young, previous metals keep here due, equal churches. Strong temperatures avoid. Established, average children could help also technical aspects. Feelings navigate now weekl Sports football 1.45 8988.48 3.75738645674136449 +AAAAAAAAEIACAAAA White, vital departments should become aga Sports football 2.88 4166.35 1.74162784631488126 +AAAAAAAAEJPDAAAA Daily, marked years may not save players. Then hot families please universally always parental opportunities. Closely medic Sports football 3.21 1605.80 0.67126045474154508 +AAAAAAAAELMCAAAA Popular, strong farms worry certainly followers. New documents will argue considerably under a men. Catholic, exist Sports football 0.10 1110.81 0.46434352081919024 +AAAAAAAAEOODAAAA Clearly great options cannot believe. Responsible products ought to condemn at a systems. Dull types assure; real ser Sports football 3.03 8226.16 3.43871958050610814 +AAAAAAAAGDEAAAAA Succ Sports football 4.47 9246.93 3.86542435967320677 +AAAAAAAAGFFDAAAA Almost busy threats go together recent sides; still tired wines shall not admit on a Sports football 3.88 7510.88 3.13971648045159802 +AAAAAAAAHAFEAAAA Economic, crude hands put available payments; irish months pay main, tropical members. Neither soft syste Sports football 4.23 2877.00 1.20265059676885365 +AAAAAAAAHAGAAAAA International, profitable schools sit rather di Sports football 81.85 205.56 0.08592869540208744 +AAAAAAAAIPLBAAAA Young features may seem actually for the plans. Unduly Sports football 9.86 3012.65 1.25935534249415605 +AAAAAAAAJLIAAAAA Standards must pa Sports football 3.63 836.01 0.34947095078370849 +AAAAAAAAJLOAAAAA Very aspects use then. Popular, weste Sports football 6.30 1501.17 0.62752276550278069 +AAAAAAAAJPFCAAAA Models may register still digital, professional birds. There necessary things can fail never irish forces. All corporate readers identify more Sports football 68.59 9190.37 3.84178100974159524 +AAAAAAAAKGCEAAAA Again sexual officials shall not Sports football 7.81 11678.56 4.88190029662873252 +AAAAAAAAKNDCAAAA Ages must answer even such as a citizens. Fatal candidates say also. Thus great friends create normally Sports football 19.60 1325.80 0.55421416795138901 +AAAAAAAALDPCAAAA Successive, joint Sports football 4.67 4363.92 1.82421654231892103 +AAAAAAAALHDBAAAA Democrats take before. Joint years woul Sports football 65.80 7674.39 3.20806733171252094 +AAAAAAAALHEAAAAA Hours take so. Now new things want common, recent drugs. Ships will st Sports football 3.32 1013.26 0.42356543054640551 +AAAAAAAALMIDAAAA Quiet, small objectives should stay as matches. In particular formal students allow then. Professional, other demands drop Sports football 1.58 2487.00 1.03962184016827912 +AAAAAAAAMDLCAAAA Super stars might like approximately stories. Major practices might allow more fresh decisions. Advanced organisations wield. Towns must not protect quickly. Active, righ Sports football 4.05 6655.69 2.78222785902276383 +AAAAAAAAMHAAAAAA Cheaply financial tales allow unfortunately safe, red meals. Who Sports football 2.91 5952.36 2.48822012727947644 +AAAAAAAAMIEBAAAA Hard figures will not help twice central principles. Collective, impor Sports football 2.33 468.64 0.19590204229049551 +AAAAAAAAMJKAAAAA Advanced, foreign stories would greet always corporate games. Recent dev Sports football 3.00 634.63 0.26528958923441696 +AAAAAAAANIEAAAAA Very questions make secret stocks. Aggressive, major years qualify for example senio Sports football 4.39 292.60 0.12231336969571310 +AAAAAAAAODBBAAAA Matters reserve more proper, concerned birds. True months result together more chemical columns. Social views reduce in a affairs. Medieval, serious sports may n Sports football 0.16 7175.77 2.99963297628642230 +AAAAAAAAOGADAAAA Proud things mus Sports football 28.70 17469.96 7.30283552990198210 +AAAAAAAAACEEAAAA Unacceptable flowers should not give reasonable, ethnic governments. Employees shall complain Sports golf 8.39 4100.46 1.45417454300510042 +AAAAAAAAAHOCAAAA Crucial products would carry silently double groups. Really full systems run usual structures. Financial departments must meet well c Sports golf 1.50 12212.90 4.33114535351326216 +AAAAAAAAANGBAAAA Different hours must not know towards a weapons. Facilities shall not know items. Today established fl Sports golf 5.73 437.77 0.15524940852766344 +AAAAAAAABEEBAAAA Educational terms must apply automatic, other objectives. Indeed financial sources pass very unacceptabl Sports golf 6.99 16143.50 5.72508126771211978 +AAAAAAAABPHCAAAA More black mothers shall repea Sports golf 14.90 7660.56 2.71671747490846200 +AAAAAAAACCMAAAAA Admini Sports golf 9.35 2840.01 1.00717242550345943 +AAAAAAAACGGCAAAA Separate, rapid bodies will start too religious surveys. Geographical, loyal things involve in order. Notes need dead for a members; at last economic managers look once more nervous skills; joint Sports golf 6.57 2341.31 0.83031498887521685 +AAAAAAAACIGDAAAA European quantities would wait Sports golf 0.73 9236.58 3.27563236818065546 +AAAAAAAACKEAAAAA Wet, suitable projects shall follow voluntarily all of a sudden resulting negotiations. High, video-taped services should not take all full eyes; wrong representatives follow royal, full figures. Fre Sports golf 3.35 7298.73 2.58839919478975935 +AAAAAAAACNHBAAAA Good, interior faces contribute with a rights. Social, certain versions pick furiously between a troops. Forward political countries bec Sports golf 7.89 4757.12 1.68705042898124194 +AAAAAAAACPMCAAAA Great, new errors w Sports golf 3.21 791.01 0.28052135742391451 +AAAAAAAADJICAAAA Stairs say long words. Newspapers will go exceedingly. Other, empty numbers must not provide therefore environmental months. Entirely bare groups buy. New days Sports golf 20.77 1505.63 0.53395199982069557 +AAAAAAAAEBEBAAAA Labour parties worry far well clear files. Finally domestic generations would not announce too; continuous, possible patterns might conceal Sports golf 4.32 2152.66 0.76341273216794201 +AAAAAAAAENPDAAAA Live processes review home at pres Sports golf 2.74 4204.30 1.49100004174076658 +AAAAAAAAFCFEAAAA Judicial models should not pick. Close dogs can refuse exactly. European, r Sports golf 5.70 6536.36 2.31803463902021193 +AAAAAAAAFIJAAAAA Pages could watch fundamental, literary components. Financial, royal elements should overcome environmental trustees. Shared areas Sports golf 3.07 4544.08 1.61149857756900853 +AAAAAAAAFOBCAAAA Demands could treat lines. Conditions suck studies. Documents could not hide local things; gold calls see together. Preferences may refuse indeed in a pieces. Old, unknown boys emerge more opposite, Sports golf 2.87 625.67 0.22188568753798383 +AAAAAAAAGBJDAAAA New sources play just. English groups evaluate here indian changes. Familiar, able authorities get direct important, emotional orde Sports golf 6.52 7170.18 2.54281061753176740 +AAAAAAAAGCHBAAAA Most angry years help intimate conditions. By far urgent police would agree Sports golf 1.81 13747.41 4.87533926785135024 +AAAAAAAAGFCAAAAA Then growing levels light sometimes human, fellow cities. Users may derive odd championships. Stages support right Sports golf 8.86 5586.76 1.98127141098295675 +AAAAAAAAGHBCAAAA Brown customers can detect too. Then human numbers cannot prepare never victorian long accountants; interests share open in the years. Full-time, underlying Sports golf 92.44 6716.33 2.38185864718140065 +AAAAAAAAGPGCAAAA Secondary, normal Sports golf 6.04 7486.01 2.65481559890393074 +AAAAAAAAHCABAAAA Wishes might behave environmental regions. Statements conflict now nuclear Sports golf 7.46 16077.73 5.70175679687386128 +AAAAAAAAHLDBAAAA Horses say. Other peasants can keep at first large kilometres. Necessarily new miles separate for an poems; interestingly indian teeth used to make further. Sports golf 3.40 752.00 0.26668697081299062 +AAAAAAAAIBLAAAAA Russians receive then definit Sports golf 8.76 20347.14 7.21584724907956645 +AAAAAAAAIDOBAAAA Independent, scientific subsidies might contain. Here certain instructions shall not imagine exhibitions. Either other attitudes buy finally. Public, right p Sports golf 4.05 198.74 0.07048054332363531 +AAAAAAAAIIAEAAAA Married professionals clarify plans. All basic children could prove more religious big trees. Sports golf 4.01 7501.44 2.66028764672260686 +AAAAAAAAIKPCAAAA Roles shall not remember primary, inc years. Young feelings can s Sports golf 5.74 3892.36 1.38037459802347363 +AAAAAAAAILFEAAAA Particular, complete artists belong much enough active cheeks; profits may see able, complete processes. Here available officials take aside at a eyebrows. Sports golf 4.07 10080.46 3.57490338005521200 +AAAAAAAAINHDAAAA Poles decide over for a managers. Properly other views include slim functions. Bright, other minutes should talk exactly certain weeks. Sports golf 6.56 1356.03 0.48089831520151552 +AAAAAAAAKBNCAAAA Inevitably dead trees establish original, primary events. Other women ought to issue almost long medical achievements. Catholic, hard cars need here difficult humans. Great, Sports golf 0.80 5928.82 2.10257851900994022 +AAAAAAAAKGHAAAAA Strong changes stay. Future claims will not recoup fo Sports golf 2.23 9989.59 3.54267752229221140 +AAAAAAAAKGMDAAAA Impressive records lie easy origins. Social schools shall bend else different details. Novel chemicals present primarily by a bags. Molecules shall see repeated Sports golf 3.63 4279.32 1.51760490417479657 +AAAAAAAAKJGAAAAA Also major pieces resign never. Substan Sports golf 4.63 55.04 0.01951921658716357 +AAAAAAAAKNCBAAAA Assets may not engage heavily always formal groups. Local, genetic offices cannot keep still sad, annual troops; supreme, natural gaps can see. Nearl Sports golf 7.20 4005.33 1.42043793192339857 +AAAAAAAAMCBBAAAA So overall investor Sports golf 2.54 15395.25 5.45972418538390139 +AAAAAAAAMDBDAAAA Brothers appoint even. Sports golf 3.65 3103.75 1.10070436922981335 +AAAAAAAAMDEAAAAA Closely substantial instructions wait for a companies; members may bring then characters; recent views should indicate near early days; objectives could not arrive categories. High gains speak Sports golf 7.73 77.67 0.02754465029660237 +AAAAAAAAMFADAAAA Neighbours shall send important, excellent games. Plain important ways note monthly, japanese figures; routinely Sports golf 4.81 616.44 0.21861238868079779 +AAAAAAAANGBCAAAA Certainly persistent players move often respective minutes; amer Sports golf 7.78 74.48 0.02641335849222279 +AAAAAAAANMHCAAAA Impossible, natural cases may wait then products. Political sectors go here sure consultants. Me Sports golf 2.14 2979.66 1.05669747267637717 +AAAAAAAANOGBAAAA Classical, small perceptions finance again ideas. Obligations determine. Clear, useful crowds could take thus formal, genetic individuals. Int Sports golf 0.68 14169.23 5.02493221735711581 +AAAAAAAAOEIAAAAA Forward working birds ought to try already then public pounds. Black, similar hands cover still at a rights. Right contracts save for example general, able feet. Systems could not t Sports golf 8.61 291.36 0.10332701571286296 +AAAAAAAAOKCDAAAA Young, severe parts must not act therefore rath Sports golf 2.17 1012.25 0.35898123165618319 +AAAAAAAAOLHDAAAA Only concerned times used to know however in the trees. Developers might not wear in the times. Studies see far variations. Calculations must not transport hardl Sports golf 0.15 8494.93 3.01261588958563618 +AAAAAAAAPEOBAAAA Sales include easier from the times. Significant, young features should not keep hardly social Sports golf 4.30 403.10 0.14295414618978261 +AAAAAAAAPJLCAAAA Likely, exciting negotiations disrupt even communications; all normal girls may think about years; allegedly old hands end darkly musical years. Individual, similar Sports golf 4.26 9885.12 3.50562860229110351 +AAAAAAAAPNFBAAAA Basic differences stem Sports golf 0.88 12915.95 4.58047284663835931 +AAAAAAAAAILAAAAA Continental issues need famous areas. Thus christian years shall agree just foreign negotiations. Sensitive centres may not assess large remains. Men eat from the ideas. Other, specific plants say Sports guns 0.19 6159.12 2.82446517920513238 +AAAAAAAAAJGCAAAA Revolutionary son Sports guns 4.83 7287.25 3.34180595233776919 +AAAAAAAACCDEAAAA Businesses may keep also behind a workers. Early, previous objectives hit wet, bottom requests. Under true hours touch similar, long sources. Widely able attitudes must appear now politica Sports guns 2.73 6762.87 3.10133441571052580 +AAAAAAAACHGBAAAA Occasional, biological questions make usually for a tools; parts will use between a machines. Languages swim alive commitments. Other russians shall finish b Sports guns 4.12 2865.32 1.31398585630415545 +AAAAAAAADDFCAAAA Again dull trials ensure suddenly; communities should produce terms. Too extra notes might choose properly social, absolute talks Sports guns 6.99 8342.32 3.82564268171208874 +AAAAAAAADKLBAAAA Only other packages shall not lift procedures. Available, only types result obviously rough parts. Deep, back boundaries assert english, blue police; findings will declare restaurants. Little, daily s Sports guns 2.81 10686.60 4.90068866722739088 +AAAAAAAADMJBAAAA Complicated, right projects forget naturally british, true weapons. Employers step also as continuous tickets. Ev Sports guns 5.02 8567.83 3.92905764075860015 +AAAAAAAAEBCBAAAA Then vague profits used to buy tonnes. I Sports guns 0.44 2445.30 1.12137199838780706 +AAAAAAAAEECDAAAA NULL Sports guns 8.03 272.49 0.12495916895296837 +AAAAAAAAEHLAAAAA Vital, possible communications go yet operational effects; Sports guns 1.48 11987.62 5.49731378371310009 +AAAAAAAAEIBBAAAA Now good properties see quite mere exceptions; long publications ought to make alone facilities. Certa Sports guns 4.20 3874.40 1.77673237253249895 +AAAAAAAAEKKBAAAA Negative patients may not get for a eyes. Past little questions perform highly only, afraid acts. Again co Sports guns 1.13 5931.38 2.72002758099107309 +AAAAAAAAEMHCAAAA Differences imagine up a feet. Tender methods shall complet Sports guns 93.05 1128.12 0.51733618730677336 +AAAAAAAAEMOBAAAA Annual communications use enough in a standards; only famous conservatives used to kill new, public children. Men dance so examples. Christian patients shall cause as busy te Sports guns 2.43 22127.23 10.14716236203600213 +AAAAAAAAENIBAAAA Courts define so. Appropriate tables surprise well to a agreemen Sports guns 7.17 131.70 0.06039532662154917 +AAAAAAAAENNBAAAA Examples should not monitor firms. Fo Sports guns 3.84 535.99 0.24579568045470114 +AAAAAAAAEONAAAAA New years can lend elements. Other, typical figures return under a flowers. Due, following others used to reject in full strong, lik Sports guns 0.78 4193.11 1.92288722862630256 +AAAAAAAAFJLAAAAA Other aspects might appear quite good Sports guns 0.21 5214.14 2.39111380676146088 +AAAAAAAAFKFCAAAA Strong chips meet to a connections; necessary, suprem Sports guns 2.74 4156.55 1.90612144926955361 +AAAAAAAAGACCAAAA Artistic children can stay significant Sports guns 5.71 4613.16 2.11551484402024129 +AAAAAAAAGDCAAAAA Old ideas must withdraw holy pensioners. Additional bo Sports guns 7.83 1028.06 0.47145041371715901 +AAAAAAAAGFCDAAAA High, capital clothes can show. Prob Sports guns 28.98 231.55 0.10618479786803121 +AAAAAAAAGMDBAAAA Settlements relocate colleagues. Well Sports guns 0.15 9689.92 4.44362857506971716 +AAAAAAAAHCLCAAAA Major, late transactions ought to determine interested, industrial group Sports guns 3.27 2963.68 1.35909203949698443 +AAAAAAAAHMCBAAAA Films exclude british, young members; spots decide other, poor agents. Black, Sports guns 7.63 834.49 0.38268258247848571 +AAAAAAAAIDNCAAAA Badly heavy reports shall keep there important, given women. Vice versa pure plants reliev Sports guns 2.78 1558.80 0.71483853559355238 +AAAAAAAAIKACAAAA Upwards new instructions help enough firms. Funds see then. Mines might play girls; odd difficulties bid complaints. Others go slightly at a fees. Empty awards find necessarily fi Sports guns 5.31 4316.40 1.97942587569669586 +AAAAAAAAIKAEAAAA Political, appointed actors might not take formal resources. Possibly new programmes might not use in a waves. Racial, suspicious reader Sports guns 1.08 15990.81 7.33310700754088619 +AAAAAAAAJLEBAAAA Golden, royal counties work then jobs. Patterns would take efficiently compl Sports guns 42.09 2480.64 1.13757830698921593 +AAAAAAAAKBECAAAA Girls help diverse, clear workers. Classes improve no longer Sports guns 3.07 147.44 0.06761341653060903 +AAAAAAAAKJICAAAA Social, large demands may attend subsequent, french sales. Small, able others will react in a principles. Enormous procedures could not move terms. Important members take so Sports guns 6.84 266.10 0.12202882622622805 +AAAAAAAAKLICAAAA Wooden, english birds say so to a states; key, video-taped trends check largely ago fast ways. Urban patients promote and so on political minu Sports guns 7.33 4309.42 1.97622496924401239 +AAAAAAAALKBAAAAA Alone, fortunate minutes can put particularly out of a consequences. Darling costs run already in a laws. Molecules discover. Temporary, political ty Sports guns 5.47 1876.47 0.86051646579755789 +AAAAAAAALKKDAAAA Good definitions deliver a bit international childre Sports guns 4.27 10401.45 4.76992384273130321 +AAAAAAAAMABBAAAA Suggestions go instead reasonable figures. More fat practices imagine Sports guns 1.92 7358.08 3.37428735692853857 +AAAAAAAAMAKBAAAA However old days hold perhaps new, gentle bones. Rules achieve also. Fine, vocational proble Sports guns 7.68 1967.40 0.90221538037384845 +AAAAAAAAMFDDAAAA Chips ought to finish. Bottles may not clear. Right, white wives used to accommodate about a words. Courts choose well new, future rewards. Permanent tourists serve ahead polit Sports guns 5.55 2717.44 1.24617066343555491 +AAAAAAAAMGMCAAAA Cold clients see lengthy, only spirits; numbers must not want once again tall leads; once naked lads make. Minutes lose front expenses. Probably alive p Sports guns 0.47 3757.58 1.72316075479575351 +AAAAAAAAMIIAAAAA Right, vital dreams vary most; documents Sports guns 4.18 2652.80 1.21652788505425697 +AAAAAAAANCGEAAAA Directly essential organisations introduce onwards atomic words. Much famous steps ma Sports guns 62.90 380.00 0.17426138281084803 +AAAAAAAAOBHBAAAA Today keen pages wil Sports guns 8.17 1181.16 0.54165940768647699 +AAAAAAAAOILCAAAA Possible roots must reveal at least upper, previous populations. So gr Sports guns 3.01 21554.07 9.88432116684688198 +AAAAAAAAOJOAAAAA Unusually global cattle shall tempt great prices. Worlds would not sign certainly deposits. Contributions predict als Sports guns 4.06 1782.00 0.81719416886560838 +AAAAAAAAPGHDAAAA In full possible products bear to a components. Lovely boards help alongside at the possibilities. True, dry papers should disagree into a c Sports guns 0.52 763.63 0.35018742041012600 +AAAAAAAAPOBEAAAA Resources go in a records. Permanent, flat applications would work Sports guns 7.43 571.34 0.26200657488197345 +AAAAAAAAAAFDAAAA Negative in Sports hockey 1.63 5985.40 2.60825063748619267 +AAAAAAAAABMDAAAA Modern facilities see; certain procedures lure for a features. Still dependent companies put little persons; procedures find to a employers. Public boards know almost also tory considerations. Sports hockey 8.87 6280.74 2.73695059793581544 +AAAAAAAAAEABAAAA Contracts will improve just by a services. Strange, educational passengers resist only english days. Difficulties should debate then impressive, linguistic applications; fine, new eyes build; roya Sports hockey 6.73 11482.83 5.00385916858448520 +AAAAAAAAAFADAAAA Following parts treat perhaps appearances. Coming studies perform loudly so professional streets. Lesser, elderly years wear equ Sports hockey 2.07 8396.19 3.65879772779683831 +AAAAAAAAAICDAAAA Girls would not enhance here inner authorities. Commercial others might not think normally problems. Loudly bright peasants see yellow candidates. Comfortable sessions may Sports hockey 5.75 3982.08 1.73526626433003944 +AAAAAAAAALPDAAAA Depen Sports hockey 3.19 1800.84 0.78474990443589989 +AAAAAAAACBICAAAA Then sophisticated numbers might not facilitate alway Sports hockey 1.14 1035.30 0.45115144935834786 +AAAAAAAACDBBAAAA Speakers get more with a Sports hockey 37.55 4112.16 1.79195107118074348 +AAAAAAAACGHBAAAA Public, available symptoms take somewhat in a minutes; nerves seem. Curious, certain islands contact again vital respects; mass rules might recognise primary, Sports hockey 8.68 334.35 0.14569930174148904 +AAAAAAAACMEEAAAA Foreign children increase about so tall leaders. Available, domestic telecommunications mess subsequently primary characteristics. Cities risk businesses. Elegant views cannot use f Sports hockey 7.88 2922.03 1.27332953691545754 +AAAAAAAADGAEAAAA All british ways trap stages. Accidents welcom Sports hockey 3.21 4828.96 2.10431015444169561 +AAAAAAAAECKAAAAA Much catholic guests invite highest problems. Long men must assume maps. Passive applications want independen Sports hockey 5.63 10772.75 4.69442845172910449 +AAAAAAAAEKMCAAAA Eyes must increase roughly. Services should love now senior, rapid sales. Sports hockey 0.88 9712.50 4.23240457055245201 +AAAAAAAAELCAAAAA International places Sports hockey 7.18 5185.35 2.25961380076336237 +AAAAAAAAEOBBAAAA Reasonable laws shall pay significant boys. Widespread operations would not run then words. Substantial paintings make stil Sports hockey 0.88 10680.29 4.65413726752387621 +AAAAAAAAFALAAAAA Military, special factors may adopt often young names. Actually large-scale workers make here advantages. Precious, odd customers study in the careers; usual women win then firms. S Sports hockey 3.48 7195.62 3.13562676715146818 +AAAAAAAAFIKCAAAA Parts work only windows. Positive, vital eyes could happen without a minds; common payments must not investigate only important seeds. Here different Sports hockey 8.94 1422.63 0.61993778267233306 +AAAAAAAAGJKAAAAA Colleagues come so; great places finish only large years. Regulations would know genuinely most other services. Opi Sports hockey 9.08 3086.02 1.34479126412522810 +AAAAAAAAHIDBAAAA Main months answer weapons. Little, norma Sports hockey 1.15 619.92 0.27014180091396407 +AAAAAAAAICCCAAAA Workers ought to widen late, close benefits. Final eyes restore yesterday high, public funds. Quickly educational days go perhap Sports hockey 3.55 11162.51 4.86427370325224722 +AAAAAAAAJBHCAAAA Then suspicious authorities can advertise perhaps important massive mammals. Easy lawyers will put. Respectively responsible pounds might acknowledge ti Sports hockey 4.00 4553.02 1.98406410891291892 +AAAAAAAAJIMCAAAA Flights might work bits. Appropriate powers ought to lie just very parental pounds Sports hockey 3.03 1200.96 0.52334091048140775 +AAAAAAAAJNLBAAAA Little hearts must not get here. Best professional hospitals achieve there foreign shoulders. Women should not forestall certainly able deals. Projects sound years. Facilities shall find dry, Sports hockey 47.20 1750.77 0.76293096010153065 +AAAAAAAAKFEBAAAA As able participants arise. As red years must make often versus a models. Alone techni Sports hockey 0.13 10294.75 4.48613096038042269 +AAAAAAAAKHDAAAAA Small regions allow so new deaths; slowly late attacks would install automatically acc Sports hockey 5.69 12283.72 5.35286205110801192 +AAAAAAAAKHEEAAAA Interesting, complete times join secure reports. Ancient, traditional markets go lessons. Rapid terms figh Sports hockey 3.26 12950.49 5.64341962078700893 +AAAAAAAAKOBAAAAA Reports may develop relevant, clear cells. Intently inc Sports hockey 7.52 1084.78 0.47271329009460889 +AAAAAAAALLPDAAAA Forces trust together from the systems. Reasons exploit even mar Sports hockey 3.36 2768.45 1.20640416302146057 +AAAAAAAALMPCAAAA Annual priests look often practical genes. Needs may n Sports hockey 0.72 2604.48 1.13495115118789706 +AAAAAAAAMBDDAAAA Tenants shall not know so realistic years. Recommendations tell. Successful, proposed actions used to link also. Holes will not become only later previo Sports hockey 5.91 6583.03 2.86867915161739080 +AAAAAAAAMEGAAAAA Then royal plans would afford certain, terrible days. Priests ought to care rarely Sports hockey 4.15 6918.52 3.01487522980268214 +AAAAAAAAMEJBAAAA Complete clubs engage to a classes; other, small estates rob sl Sports hockey 8.86 2201.70 0.95943218975395970 +AAAAAAAAMFBBAAAA Details accompany ok. Black savings go ju Sports hockey 7.28 15049.92 6.55828573430617849 +AAAAAAAAMGAEAAAA Issues recognise only previous Sports hockey 75.67 4488.20 1.95581757462584454 +AAAAAAAAMHLBAAAA Very old efforts bring sorry supporters. Almost other subjects sha Sports hockey 1.96 7640.40 3.32944801862022696 +AAAAAAAAMIKCAAAA Too female dates will achieve also national, capable statements. Actual, small lights see then cheap effects. Free peasants used Sports hockey 3.59 8586.28 3.74163302095681932 +AAAAAAAANACBAAAA As national managers shall respect years. Other police could not consider. Therefore true bodies continue in the factors. Special relations would reach on Sports hockey 3.94 1856.04 0.80880434276737946 +AAAAAAAANMPDAAAA Tonight certain authorities hang with a cattle. Internationa Sports hockey 0.61 9094.17 3.96295564204694903 +AAAAAAAAOBHDAAAA Psychological, ill activities talk rather right windows. Leaders would know adequately sacred, ordinary offenders; important minutes could affect again norma Sports hockey 7.66 794.92 0.34640134272571996 +AAAAAAAAOCECAAAA British observations speak great quantities. Personal, ready th Sports hockey 1.66 274.86 0.11977541521359557 +AAAAAAAAOJFBAAAA Late, chief standards guarantee publicly police. Also political years might come curious years. Systems may not follow so with a times. Central, silent towns must apologis Sports hockey 40.41 5501.55 2.39740389859694645 +AAAAAAAAAFGAAAAA Columns blame rapidly. English users may not get excellent, female manufactu Sports optics 0.25 1588.38 0.64760161773605996 +AAAAAAAAAGABAAAA Softly old women ask perhaps as a questions; relevant needs used to fall. Entries would not call together questions. N Sports optics 3.85 6270.40 2.55651744787279515 +AAAAAAAAALMAAAAA Projects mount in general perhaps busy things. Accounts will fail. Often d Sports optics 56.35 1751.04 0.71392005484868258 +AAAAAAAAAPEBAAAA Good duties cannot determine gifts. Today social others succeed really quick eggs. Asleep, liable observers understand more after a operations. States must wish just similar women. Questio Sports optics 4.66 2203.00 0.89818957923956490 +AAAAAAAABKPCAAAA Solid police must lift increasingly western girls. However central days choose widely over a drivers. Able years release commonly christian, aware muscles; sometimes important Sports optics 2.47 24705.19 10.07260291018316218 +AAAAAAAACCFBAAAA Mad, social circles could arrive increased eggs. Shareholders search very low carers. Fast, significant patients will not seize then capital memorie Sports optics 1.38 6498.54 2.64953286803063189 +AAAAAAAACEIDAAAA Obvious eyes talk lives. Neutral, real guests must stay in a departments. Hands can drop in the rounds. Flexible, mutual margins may pass like women; large clubs try. Old, sure records would Sports optics 6.07 1813.00 0.73918189158480761 +AAAAAAAACMJDAAAA Circumstances join by a members. Human, personal priests will not obtain again wide, statutory days. Whole, new kids shall not encourage Sports optics 4.53 6033.35 2.45986931362007665 +AAAAAAAACODDAAAA Nurses should see certainly eyes. Clubs shall go individual procedures. New, internal police might read too international children; healthy, sufficient years break well only new agent Sports optics 8.75 9654.45 3.93623530789351671 +AAAAAAAADIDCAAAA Identical solicitors must maintain sources. Factors take already unusual minutes. Just various sales sell agricultural, long states. Sports optics 3.77 1573.11 0.64137585519634677 +AAAAAAAADJDEAAAA New hotels join increases. Agencies might not prov Sports optics 40.19 2052.76 0.83693492541071686 +AAAAAAAAEAFCAAAA Aware, single times would ring to the men. Again double months cover that. Accurate politicians send so social hotels. Other, urban feelings upset just wild eyebrows. True, magnificent p Sports optics 3.24 642.52 0.26196312685111450 +AAAAAAAAENJAAAAA Other, international colours s Sports optics 3.14 11101.71 4.52630060541973219 +AAAAAAAAGFADAAAA Quick artists must hope tough teachers. Social conflicts find rapidly from a shareholders; other tools Sports optics 3.81 10100.29 4.11800963472427822 +AAAAAAAAGFECAAAA New, able officers may believe often. Losses weep fast excellent, old hours. Able, only regulations shall not let by a countries. Dreams back a little. Sophisticated, Sports optics 8.41 1446.65 0.58981659319424265 +AAAAAAAAGHPBAAAA Acute, serious forms change just premises. Above causal buildings may pay so open, traditional consequen Sports optics 4.49 7490.92 3.05413812206865251 +AAAAAAAAGMKDAAAA Ago sexual courts may attract. Important, alone observations expect. New, available ways represent years. Excell Sports optics 8.59 3179.49 1.29631628928570322 +AAAAAAAAHKEEAAAA Bombs shall not help. Angles pull sometimes. Measures train still african pictures. Teachers wear by the motives. Attractive months shall give Sports optics 0.92 NULL NULL +AAAAAAAAIGKBAAAA Other, different problems spread importantly only likely commitment Sports optics 3.10 8596.18 3.50476590888223467 +AAAAAAAAIGNCAAAA Possible opponents can inform also foreign, new heads. Losses face most qualifications. High difficulties will not walk results. Direct, ou Sports optics 0.27 149.24 0.06084694180922046 +AAAAAAAAIIPDAAAA Drugs hold years. Cells might reconsider now. Wrong players meet too rapid, integrated parents. Complete, social women used to includ Sports optics 4.94 13154.62 5.36329668763339318 +AAAAAAAAIJHCAAAA Holidays will find soon so international expectations; furious children would not talk in order reasons; there current stones shall give as firms. Central drugs ought to love european, following Sports optics 9.08 13906.80 5.66996951455686841 +AAAAAAAAIJJCAAAA European nights accompany however expensi Sports optics 1.37 3255.97 1.32749810454682075 +AAAAAAAAILPDAAAA Earnings used to connect of course. Only big branches show into the men. Tiny trousers mediate. Highest proposed m Sports optics 8.81 3903.78 1.59161802798176516 +AAAAAAAAKBPAAAAA Wild, other services change less at a hours. Inherently southern days would win almost remarkable, separate firms; strong, professional children might damage other fea Sports optics 1.25 10597.58 4.32076074496487887 +AAAAAAAAKDJDAAAA Industrial, sexual minutes must cure crowds. Sports optics 3.33 503.37 0.20522999931993635 +AAAAAAAAKHPAAAAA Sad recordings will borrow most long teachers; then bold shares show markets. Common, dark skills watch really to a le Sports optics 8.63 838.35 0.34180537165478404 +AAAAAAAAKKJAAAAA National, little grounds must not hate broadly. Teachers define abroad normally tall researchers. Cultures handle centres. Major addresses used to look Sports optics 1.61 12110.40 4.93755564249787867 +AAAAAAAAKLKDAAAA Excellent, difficult relations attempt. Boots dismantle really social sheets. Literary sp Sports optics 1.67 2628.08 1.07149980454285779 +AAAAAAAAKLPBAAAA Obvious clubs should finance at leas Sports optics 5.51 1283.02 0.52310267542258128 +AAAAAAAAMAICAAAA Alleged books ought to go altogether different databases; artists will listen years. Forward cold others check effectively. Quite numerous d Sports optics 5.42 3201.52 1.30529818507809887 +AAAAAAAAMDGBAAAA Teams judge conscious shareholders. Else local areas imagine ea Sports optics 2.39 6080.10 2.47892985053766615 +AAAAAAAAMFPAAAAA Tall students should encompass much true women. Rough birds ought to protect as possible families. Political, dead proceedings Sports optics 1.06 5878.74 2.39683295826545608 +AAAAAAAAMMJCAAAA Natural, political manufacturers must not pr Sports optics 2.60 1879.45 0.76627435528906048 +AAAAAAAANFCCAAAA Physical, nationa Sports optics 52.14 5315.52 2.16720139457080890 +AAAAAAAANFDEAAAA Rules share briefly ago specific subsidies. Maybe new subjects should scor Sports optics 1.12 NULL NULL +AAAAAAAANHEEAAAA Exchanges see with a costs. Possible controls achieve yet high similar machines. Rights would not sum suit Sports optics 4.85 337.31 0.13752534134057995 +AAAAAAAANIBDAAAA Legal, local prices ask central instruments. Structures cover for a parents. International tourists should Sports optics 1.84 3778.91 1.54070702809086890 +AAAAAAAANJMDAAAA Wings can go yellow, expected eyes. Sports optics 8.93 5543.20 2.26002926719961695 +AAAAAAAANPBCAAAA Hot grounds shall pass. Impressive methods could change very basic voices. Concrete, desirable centres pay again in a ingredients. Positio Sports optics 1.04 2610.25 1.06423029923289799 +AAAAAAAAOBOCAAAA Small aspects can allow obvious, redundant colours. Past, sound individuals give both; soft, religious months improve; customers use once for a fore Sports optics 0.82 1475.16 0.60144046287382504 +AAAAAAAAOJFAAAAA Injuries answer so good issues. Aside aware definitions m Sports optics 1.71 6407.03 2.61222314111451179 +AAAAAAAAOMIAAAAA Scenes should not learn. Magistrates produce somewhat on a businesses; extremely national values see everywhere. Northern engines shall not aim; rom Sports optics 1.88 6498.82 2.64964702739612762 +AAAAAAAAONGCAAAA Colonies give. Even formal payments may follow comparative, frequent years. Perhaps residential messages face times. Late houses talk then conditions. Officials may includ Sports optics 76.62 15211.44 6.20188692384379802 +AAAAAAAAPBPDAAAA Great structures should not survive even here various areas. Cultural results choose likely, female hours. Gold feelings ou Sports optics 9.72 3879.70 1.58180032254913297 +AAAAAAAAPKMDAAAA Social cases need. Inc, right products can know states. Whole, economic years should run relatively new notes. Markets can stop just keen words. Now common services abuse only new, narrow feelings. Ye Sports optics 0.97 8141.82 3.31951787564424615 +AAAAAAAAAAEAAAAA Only economic shares last too white patients. Ever environmental markets might come slightly w Sports outdoor 1.07 1920.21 0.69563739953531432 +AAAAAAAAADPCAAAA Strict results wonder indeed ago possible factors; wrong tables survive for example known differences. Featur Sports outdoor 3.18 7506.80 2.71949986242738947 +AAAAAAAAAHADAAAA Total, happy arrangements control indeed. Particularly internatio Sports outdoor 4.20 5584.92 2.02325746945009538 +AAAAAAAAAJMBAAAA Easy, local stages may not get elected, alone pages; clean mem Sports outdoor 1.93 11116.50 4.02719137590905246 +AAAAAAAAALFEAAAA Public questions call under way far essential taxes; Sports outdoor 1.23 9780.48 3.54318937689479327 +AAAAAAAAALGAAAAA Preliminary, central jobs would attend unhappily personal members; as blue duties must sound remaining, slow voices. Bad years can seem short drugs. Major problems fit more middle countries. S Sports outdoor 3.62 276.60 0.10020430302491287 +AAAAAAAAAOOBAAAA Houses decide quite. Elements cannot assume simply; simple, cruel days could know. Sports outdoor 7.17 NULL NULL +AAAAAAAABAKCAAAA Principles take hardly perhaps financial women. Men revive so in a classes. Only domestic miles perform relations. Urgent, male developers relax major po Sports outdoor 2.50 7845.25 2.84211065909688245 +AAAAAAAABBCDAAAA Costs use again successfully coming weeks. Processes can stress less heavy, oral issues. Personally cheap officials shall go current events. Natural parties imagine powerfully without the we Sports outdoor 4.07 3610.83 1.30810088030168523 +AAAAAAAABIDEAAAA Ago natural taxes could protect rats. More local days shall tend closely. Proteins may intervene very perfect men. Procedures make expens Sports outdoor 8.79 12330.06 4.46682960432160944 +AAAAAAAABIKAAAAA European Sports outdoor 29.44 11343.15 4.10930021640289375 +AAAAAAAABOEAAAAA Numbers choose special bodies. Main pictures offset like a changes; beautiful, large elections must suspend. Electronic p Sports outdoor 5.79 6902.40 2.50054295444381268 +AAAAAAAACBKAAAAA Yet green experiments think wonderful minutes. Scottish years may remove twice parental features. Good boundaries look please. French, e Sports outdoor 8.75 3992.78 1.44647048818442374 +AAAAAAAACFMAAAAA Good products may say pp.. Substantial, front flats become actually. Bills tr Sports outdoor 9.06 3258.39 1.18042190503740363 +AAAAAAAADCMCAAAA Modern personnel would keep Sports outdoor 0.48 6309.82 2.28586809585197296 +AAAAAAAADFGCAAAA Initial, real signals keep perfect, free sectors; just funny deposits can understand sufficiently. Entire relations shall not relate; poor views must reach probably. Years Sports outdoor 2.66 17724.56 6.42110333052512525 +AAAAAAAADPBDAAAA Unacceptable events must not persuade at least but for a companies; horses would try also crude skills. Turkish, new animals go further scottish lands. European elements believe Sports outdoor 9.19 702.52 0.25450298973630437 +AAAAAAAAEDGAAAAA Eyes should jump rapidly closer explicit things. Green, radical children could ensure middle consumers. Likely minutes think very pa Sports outdoor 2.37 8733.77 3.16399615195189179 +AAAAAAAAEDNCAAAA So competent candidates would enter suddenly almost cold situations; eyebrows could read enough rational sales. Impossible Sports outdoor 0.33 2072.27 0.75072440719246635 +AAAAAAAAEHHCAAAA However subsequent steps share terribly existing communications; less great responsibilities speed at all long-term mountains. Of Sports outdoor 4.39 3486.57 1.26308502096012459 +AAAAAAAAEIPBAAAA Industries give much proposals. Possible, strong goals ought to live most total criteria. The Sports outdoor 96.84 5462.95 1.97907121189424352 +AAAAAAAAEJIBAAAA Only single galleries discover in the countries. Clean front products ought to shoot even. Ready, educational questions ought to sense shortly tests. Sciences stop. Upright variou Sports outdoor 1.53 1332.46 0.48271231239542806 +AAAAAAAAELICAAAA Economic elements used to hear as Sports outdoor 0.40 396.48 0.14363341309948465 +AAAAAAAAEMBCAAAA Social, joint functions should suit. Best absolute goods might not lose still western wonderful hundreds. Inches feel certain years. Diverse lives put breasts; very good police shall Sports outdoor 5.91 1973.74 0.71502979411565989 +AAAAAAAAEOIAAAAA Trees work Sports outdoor 3.30 8407.66 3.04585578586565052 +AAAAAAAAFHKAAAAA Steps cannot stay only able transaction Sports outdoor 6.89 702.30 0.25442329000143278 +AAAAAAAAGLMAAAAA Stars divorce there s Sports outdoor 2.51 7314.38 2.64979157613652275 +AAAAAAAAGMCAAAAA Original women shall know here necessarily national goods. Accounts will make as. Independent members will find a little dreams. Short jobs assist widely new moments. Ago passive represen Sports outdoor 9.83 5957.43 2.15820723416379853 +AAAAAAAAGNEDAAAA Distinctive things used to pick today symbolic pictures. Helpful lips know still. Concerned theories must accommodate very in the ph Sports outdoor 27.94 9643.98 3.49373931412219527 +AAAAAAAAHIEAAAAA Even short boards can expel anywhere secure charming details. Specia Sports outdoor 6.91 8327.04 3.01664945575043550 +AAAAAAAAIDAAAAAA Ideas form on the needs. Firstly rough operations might begin worldwide obvious activities. Twins Sports outdoor 4.30 2362.14 0.85573605331622446 +AAAAAAAAIDADAAAA Creative teachers may close concerned, foreign parts. Alone desirable fires put pupils; areas begin behind a countries. Kindly able rates lead employers. Songs point thoroughly; large, acute others sa Sports outdoor 2.27 10905.96 3.95091872963694416 +AAAAAAAAIMEBAAAA Obviously base children must seem most for a years. Just available Sports outdoor 5.16 5010.90 1.81530637030924041 +AAAAAAAAIMNCAAAA Always small authorities make after a nations; forms will retrieve now. Financial, giant words render american, sensitive activities. Written eggs might not grant now really existing entries; grounds Sports outdoor 6.44 4934.08 1.78747667197817097 +AAAAAAAAJNIBAAAA Apparently realistic minutes see. Ful Sports outdoor 2.79 3360.22 1.21731201413728388 +AAAAAAAAJPEAAAAA Less social teeth play instead as social children. Advances mean very now slow bases. Small fit managers must think about sites; full, civil weap Sports outdoor 96.73 8555.01 3.09923649465350631 +AAAAAAAAKFACAAAA Moreover overall miles say. Leaves may order faintly sure trees. Political, certain drinks protect to a parents. New minutes remember satisfied, exciting feet. Cri Sports outdoor 5.71 3006.51 1.08917295403987994 +AAAAAAAAKHGDAAAA Alone healthy sales might meet far other roots. French groups look up to a workers. Fully average miners would walk inadequate considerations. Small, sure goods may admire more app Sports outdoor 0.48 1427.56 0.51716433415128205 +AAAAAAAAKJBCAAAA True champions get all the same police. Especially clear issues move further great homes. Better environmental sessions burn. Bonds shall test already elderly areas. Imperial, close schools press Sports outdoor 1.71 724.38 0.26242224521036292 +AAAAAAAAKMNAAAAA Public, great addresses must prefer thick maybe dangerous problems. Public pages may shoot now injuries. Flat groups know rather special responsibilities; nuclear months can see dou Sports outdoor 9.74 6478.02 2.34680216587652229 +AAAAAAAALEDEAAAA Quite significant levels move chiefly dirty, actual beliefs. Away significant views bury. Practical proceedings build a bit. Funds think about prime s Sports outdoor 9.44 3562.95 1.29075531982145086 +AAAAAAAAMAHBAAAA Independent, different attitudes include greatly other, bottom waters. Twin others should exert. Extraordinary, bottom tables could go only results. Good, early pupils shall say per Sports outdoor 98.21 5097.92 1.84683123816617431 +AAAAAAAAMFEEAAAA Theories must not Sports outdoor 0.92 453.25 0.16419956741157541 +AAAAAAAAMFKCAAAA Great, possible children used to Sports outdoor 4.00 8014.65 2.90347945494800407 +AAAAAAAAMJBEAAAA Truly growing visitors shall not receive open, personal times. Large societies Sports outdoor 12.35 2130.34 0.77176151448334375 +AAAAAAAAMNBAAAAA So Sports outdoor 2.12 6574.51 2.38175774504815585 +AAAAAAAAMNFEAAAA Very major companies would not remedy ever future, clear movies. Famous, equal fees know open, active rights. Original hours apply so. Social, technical rates could Sports outdoor 3.18 1551.09 0.56191573528167788 +AAAAAAAAMOJDAAAA Social thousands choose especially blue claims. Social, right professionals can go tons. General projects must ma Sports outdoor 0.64 1598.82 0.57920695503359072 +AAAAAAAAOBJCAAAA Prominent, regional tonnes ought to replace extremely. Women could make very young, equal hours. Q Sports outdoor 4.73 NULL NULL +AAAAAAAAOELDAAAA Most whole councils arise already so social customers. More sc Sports outdoor 2.11 1583.53 0.57366782346001546 +AAAAAAAAOGCAAAAA Various pockets can get. Areas conduct photographs. Ever Sports outdoor 1.85 1513.96 0.54846459366448694 +AAAAAAAAOHACAAAA Scientific risks would use. Quiet minutes imagine times; arms cut inner appeals. Areas happen straight in a changes. Fears kick very currently silent Sports outdoor 4.22 474.41 0.17186523282013346 +AAAAAAAAOKHAAAAA Clothes realise almost necessary females. Foreign, cultural others may give bad ya Sports outdoor 7.21 4335.56 1.57064992054479841 +AAAAAAAAOKIDAAAA Heavy years could come much through a genes. Dealers come so sincerely educational characters. Studies must handle Sports outdoor 2.12 7347.30 2.66171755464548924 +AAAAAAAAOKOCAAAA Various, personal benefits must not remember at le Sports outdoor 0.34 6983.49 2.52991955217443519 +AAAAAAAAONOAAAAA Losses try a little cho Sports outdoor 4.86 1698.82 0.61543410724794823 +AAAAAAAAPKLBAAAA Industr Sports outdoor 8.35 1902.72 0.68930127061302319 +AAAAAAAAPOPCAAAA Nearly cultural sheets might decide to a years. Loudly new marks create lives. Local, new arrangements must not face b Sports outdoor 1.39 431.65 0.15637450253327419 +AAAAAAAAAGBBAAAA Also religious bits might hear so extensive western talks. Sometimes complete settings mean also minutes. Other, available theories admit both just old years. Considerable seconds will prepare che Sports pools 0.62 10914.03 4.26659608077049963 +AAAAAAAAANEDAAAA Other sports take prime tables; sources think in a priests. Fine, key eyes keep always important branches. Still local effects shall get much; black, final metho Sports pools 2.25 1716.96 0.67120713492996785 +AAAAAAAAAOJAAAAA Factors would impose that is free, liable thoughts; significant wives buy useful sports; russians make nearly outstanding animals. Problems write. Finally per Sports pools 2.04 10920.36 4.26907065278388765 +AAAAAAAAAPEDAAAA Popular systems associate evenly public rights. Unlike mothers experiment around languages. Chea Sports pools 8.52 3232.70 1.26375180848016674 +AAAAAAAABDBCAAAA Subsequent feet can accept regardless. Individual, following arms hold prime officials. Assistant acids might not get however necessary times. Sometimes new times shall not take about. Small Sports pools 1.90 9375.14 3.66500143216343934 +AAAAAAAABNOAAAAA Bonds will set ever into the nations. Distinguished, philosophical employees may not include. General, existing tiles must continue only quiet missiles. Small ve Sports pools 12.34 9502.98 3.71497762271502301 +AAAAAAAACAIDAAAA Western products become grea Sports pools 8.19 12699.99 4.96477722342934165 +AAAAAAAACGOBAAAA Very old circumstances explore fairly upon a lines. Crucial, active looks mean alone bloody recordings; poor bacteria could not transfer both at a properties. States could not understand really at a Sports pools 3.35 2713.46 1.06076653640566500 +AAAAAAAACIOCAAAA Years ought to know then. Associated, simple activities would not indicate now for a brothers. Workers get organizations. S Sports pools 20.43 4211.26 1.64629796794635660 +AAAAAAAACJCBAAAA Supreme injuries could think conditions. Basic, eventual c Sports pools 9.13 3177.04 1.24199277557887491 +AAAAAAAADHCBAAAA Able systems merge from a areas. Most chief efforts must find never for the time being economic directors. Activities sit there. Available polic Sports pools 3.10 4811.17 1.88081937340474643 +AAAAAAAAECEBAAAA Carers get m Sports pools 5.77 4684.53 1.83131229603105623 +AAAAAAAAEEFBAAAA Privileges cut perhaps reasons. Ideas finish times. Women envy general programmes. Hands shall unveil never to a facilities; official proposals conform. Scot Sports pools 7.52 8558.76 3.34585591868955110 +AAAAAAAAEIJBAAAA Central, clear awards announce. Single, very proposals help dry maps. New questions Sports pools 2.90 2934.22 1.14706772403213253 +AAAAAAAAFBDEAAAA Able troubles dust into the styles. Independent feet kill wounds. Fundamental months should exploit arms. Massive years read only modern courses; twin forms shall become products. Even h Sports pools 6.81 6802.61 2.65932832922487921 +AAAAAAAAFICBAAAA Far good grounds change clearly rocks. Growing, Sports pools 1.99 5753.89 2.24935468595785151 +AAAAAAAAFPBBAAAA Secret, familiar questions ought to influence historical values. Central, net investors can hope. So chief arrangements shoul Sports pools 6.13 4628.51 1.80941252917639637 +AAAAAAAAGCFEAAAA Fine, high letters see now suddenly prime forces. Things used to know temporary men. Late, special methods provide fr Sports pools 2.85 2565.78 1.00303434131290940 +AAAAAAAAGCPAAAAA Directors could involve. No longer local patients see waste lovers. Only direct aims canno Sports pools 60.43 1100.10 0.43005950583383284 +AAAAAAAAGEKDAAAA Similarly direct changes can alienate men; ways surrender forms. Players must develop deep. Social, serious thousands walk. Thanks will not say organisations. Natur Sports pools 3.39 3166.29 1.23779030336024597 +AAAAAAAAGJEBAAAA Simple, environmental rights ought to detail thick disabled days; also old drinks move to a conditions. Sports pools 8.46 825.24 0.32260913243733498 +AAAAAAAAGNDDAAAA Previous, significant flats give all formally co Sports pools 2.82 6467.74 2.52841838765722572 +AAAAAAAAGNECAAAA Dangerous, other ladies may know neatly. Effortlessly growing services might encourage in the citizens. Banks use secondly other, similar responses. Indirect branches shall not buy i Sports pools 4.74 1246.28 0.48720530945422161 +AAAAAAAAGOJDAAAA Literary, sensitive pages could not know now; very public program Sports pools 3.36 2399.19 0.93790970439184930 +AAAAAAAAGPKAAAAA Christian, red laboratories prevent; shoes allow most to a positions. Now religious passengers will not know always in a elections. Southern ages abandon northern terms. Thoughts go as Sports pools 2.22 6752.13 2.63959430154149417 +AAAAAAAAHNMCAAAA Things used to reappear. Good powers lead. Rare, traditional months may pay too. Shows tend anywhere extra pp.; canadian, proper questions can investigate only small, certain countrie Sports pools 4.95 478.95 0.18723479712672870 +AAAAAAAAIEGBAAAA Like records start clear, likely un Sports pools 0.52 127.98 0.05003092042233790 +AAAAAAAAIHIAAAAA Problems might introduce therefore now public details. Early future children shall annoy ever sharp services; civil lines must fly. Finally other serv Sports pools 4.38 3165.54 1.23749710762406255 +AAAAAAAAJNFEAAAA Exclusive, different friends find for the features. Procedures comprehend totally ey Sports pools 3.90 7853.37 3.07009946489432581 +AAAAAAAAKGFCAAAA Direct, different traders woul Sports pools 4.53 3602.83 1.40844585892492317 +AAAAAAAAKIGBAAAA Southern hours see Sports pools 7.73 2352.82 0.91978238934274937 +AAAAAAAAKJEAAAAA Unable centuries may think away individuals. True, additional feet appear generally recent, pri Sports pools 3.10 741.45 0.28985330479092388 +AAAAAAAAKOABAAAA Basic levels look early, video-taped rights. Employees might not prevail later. Causal, permanent arms could not know here public vessels Sports pools 13.28 4827.92 1.88736741151284270 +AAAAAAAALGOAAAAA Thus aware parties would conduct either at the poems. Things plan. Instead old organizations should show rather necessary, b Sports pools 77.38 4657.72 1.82083152578161976 +AAAAAAAALMEBAAAA Thoughtfully fine Sports pools 4.43 6849.91 2.67781920698684657 +AAAAAAAAMANAAAAA Types can scratch like a Sports pools 9.69 3733.27 1.45943846136194267 +AAAAAAAAMGHDAAAA Only sexual functions would avoid special pati Sports pools 8.64 4120.56 1.61084083025057563 +AAAAAAAAMJACAAAA Still male versions will get in a colonies. Wide wages would com Sports pools 1.46 5664.01 2.21421810893363108 +AAAAAAAAMLDDAAAA Then available police rememb Sports pools 0.40 1103.32 0.43131829286118030 +AAAAAAAAMLMAAAAA Pressure Sports pools 5.42 3879.88 1.51675236387107660 +AAAAAAAAMPDCAAAA Consumers remind related, slight customers. Large purposes like with a systems; types must go programmes. Main followers shall reduce al Sports pools 15.70 1464.58 0.57254481506600755 +AAAAAAAANHHBAAAA Final holes agree really probably clear children. So good feet must imply birds. Newly british forces ought to raise nevertheless supreme, fine problems. Necessarily good units may push only Sports pools 2.20 1319.87 0.51597367508853827 +AAAAAAAANNFAAAAA Men make only. Flat, distant depths would assert local, Sports pools 7.24 10909.61 4.26486818056525871 +AAAAAAAAOCGAAAAA Apparently other offenders should approach Sports pools 0.36 15958.64 6.23867360438145453 +AAAAAAAAODLBAAAA Workers relieve fast quite female photographs. Other, automatic shares want away right games. Sports pools 1.82 3069.94 1.20012442445188328 +AAAAAAAAOHDCAAAA Here ready critics stay services. Excellent years ought to Sports pools 55.17 2208.60 0.86340280391291993 +AAAAAAAAOHOAAAAA Never future depths c Sports pools 23.19 4555.50 1.78087090157806155 +AAAAAAAAOLIBAAAA Real ships suspend for instance worth the arms; ago econo Sports pools 3.46 38.42 0.01501944024555573 +AAAAAAAAOODDAAAA Famous, busy shoes will not secure. Dark, extraordinary thousands might not look then. Numbers ought to e Sports pools 6.47 7750.63 3.02993555831368042 +AAAAAAAAPJDBAAAA Massive, military measures must get standards. Services make as well fine Sports pools 0.51 10656.29 4.16583838871194852 +AAAAAAAAPLCDAAAA Critics shall not print still black parents. Multiple, accessible responses exclude against a areas. Mo Sports pools 6.14 4995.43 1.95285170187028778 +AAAAAAAAPLIDAAAA Forces eliminate away. New, large characteristics should reconsider right, used firms. Peculiar principles establish degrees. More growing arts may not say about. Actual animals move here Sports pools 2.65 1461.99 0.57153231245705415 +AAAAAAAAPOBBAAAA Senior disputes can bring tonight controversial houses. Heavy, real examples should not offer nearly free effects. Worlds will not add. Agricultural, rare defendants draw maybe possibl Sports pools 3.45 7092.42 2.77262307096263314 +AAAAAAAAAFCEAAAA Free plans ca Sports sailing 0.98 6984.42 2.34770798957927730 +AAAAAAAAAOFBAAAA Special thousands take so reforms. Finally reliable acids used to go pale, small days; great, foreign judges show vice versa fair, true arrangements Sports sailing 0.90 11949.72 4.01671908579886112 +AAAAAAAABAFEAAAA References should make private women. Additional, northern values ar Sports sailing 0.63 14040.42 4.71947652218060722 +AAAAAAAABFJBAAAA More critical photographs balance just now serious values. Scottish, practical views suppl Sports sailing 5.19 2863.69 0.96258642703020159 +AAAAAAAABLHDAAAA Quite british tonnes could buy successfully surprising processes; local interests used to suggest suddenly other solicitors. Shares return just real, royal companies. Crucial, old groups study. Child Sports sailing 95.70 6541.62 2.19886741329868364 +AAAAAAAACDEBAAAA Then other rates may make more at once above councils. Camps could give Sports sailing 0.61 8648.26 2.90698284151853421 +AAAAAAAACEEDAAAA Scottish, british colleagues enable about a workers. Most good persons could read with a years. Indeed specific damages believe organisations. Immediate facilitie Sports sailing 1.74 7276.84 2.44600058514380124 +AAAAAAAACLDEAAAA Easy, natural leaves contin Sports sailing 1.73 12739.66 4.28224556463149924 +AAAAAAAACLFBAAAA New routes cannot test over a others. Armed, brown fans make so in a techniques. Electronic, subsequent professionals used to follow in a matters. Enough substantial standards Sports sailing 3.07 5349.42 1.79812727092803377 +AAAAAAAACNFDAAAA Open times ought to add actually soviet attitudes. Women must imagine of course inner streets. Rightly big records enable yesterday st Sports sailing 6.43 2470.80 0.83052234840580583 +AAAAAAAACOPDAAAA External, definite securities might know then particular others; always local years must buy right children. British effects used to enable powerful, Sports sailing 5.35 NULL NULL +AAAAAAAADFOAAAAA Important, broad investors can see dearly vulnerable troops. Eastern, poor lists need genuine facilities. Figures meet equally children. Other, defensive changes go old, new companies; Sports sailing 71.43 17348.99 5.83160268628332577 +AAAAAAAADOIAAAAA Young, black boys spread too wealthy, major numbers. Profitable drawings might think better purposes. Industr Sports sailing 3.24 12918.54 4.34237339273690257 +AAAAAAAADOODAAAA Joint texts take only local, asleep shareholders. Detailed courses fast programmes. Soft students know settlements; just b Sports sailing 4.70 1007.64 0.33870306748730216 +AAAAAAAAEAGEAAAA Only american aspirations will not provide then on a subjec Sports sailing 9.32 2524.02 0.84841145289915090 +AAAAAAAAECFAAAAA Equal songs will overcome slight contracts. Large, inner odds go even good women. Feet could not find hard strong models. Bloody machines see dark heads. Huge, only men make at the advis Sports sailing 2.07 2504.57 0.84187362722467586 +AAAAAAAAEJPBAAAA Prisoners raise both. Medical children sell; activities Sports sailing 1.25 8453.80 2.84161803017362852 +AAAAAAAAELBEAAAA Benefits may hold Sports sailing 8.02 5687.08 1.91162661371688936 +AAAAAAAAEMLCAAAA Ethnic positions must buy years. Other efforts should get; common goods show exactly aware eyes; foreign, unfair fans may carry thus daily, national actions. Sports sailing 4.63 4728.78 1.58950844693799842 +AAAAAAAAENACAAAA Criteria shall announce far about other waves. Farmers see possibly; just english managers clean. Head files see both. Comparisons may n Sports sailing 4.18 1308.47 0.43982255836916981 +AAAAAAAAEOEBAAAA Connections present high secondary benefits. Levels could compete. Psychological students ought to wonder advanced seats. Of course rich functions would see items; unlikely id Sports sailing 9.39 2534.25 0.85185011390942748 +AAAAAAAAEOJBAAAA Well bad areas seem Sports sailing 0.39 2413.53 0.81127189717818704 +AAAAAAAAEPOCAAAA Blue, united ministers know childr Sports sailing 4.68 530.93 0.17846415348838210 +AAAAAAAAFGBBAAAA Dear, continuous problems Sports sailing 5.90 8982.06 3.01918470322237831 +AAAAAAAAGBNDAAAA Prices acquire more out of a christians. Efficiently local prices Sports sailing 2.11 8027.95 2.69847494207721747 +AAAAAAAAGOPCAAAA Good, capable studies might like bad apparently new years. Modest, payable plants could feed there english women. New, local recommendations last public novels. Candidates must save as orange pla Sports sailing 4.28 1617.69 0.54376222186845881 +AAAAAAAAHHHBAAAA Mothers may not obtain p Sports sailing 9.99 205.80 0.06917658220087212 +AAAAAAAAICFEAAAA British figures can tell much white methods. New, french men could think marginally nuclear relatives. Electronic, differ Sports sailing 7.39 13316.13 4.47601730584304808 +AAAAAAAAIEJCAAAA Real appearances could join miles. A Sports sailing 2.44 1182.16 0.39736534700963551 +AAAAAAAAIJIDAAAA At present financial areas used to link very purposes. Likely members can retaliate true, blac Sports sailing 1.69 7800.18 2.62191347401165555 +AAAAAAAAIKLCAAAA Special birds will not answer especially then public walls. Most human areas could require major groups. Particularly diverse children could continue to the readers Sports sailing 4.71 7976.59 2.68121104867664997 +AAAAAAAAIPPAAAAA Students would rise broad obligations. Good, statistical children would not see. Gradually elegant cases can look w Sports sailing 4.63 391.82 0.13170441417855061 +AAAAAAAAJBADAAAA Reliable stages cannot see similarly. Feelings repeat together significant, available notes. Rich, basic roots provide instinctively before the talks. Parties arrest there other investigations. Bom Sports sailing 7.89 7983.29 2.68346315315063365 +AAAAAAAAJKOBAAAA Demands can imagine also purely fresh eyebrows. Busy skills become almost; complete pa Sports sailing 4.98 12443.47 4.18268574013161433 +AAAAAAAAJNPAAAAA Proper applications stand now very limited arms. Angrily slow boys shall aid too previous old masses. Mechanical contents think through the times. Sequences may not agree. Old, working stren Sports sailing 0.63 679.89 0.22853482250996573 +AAAAAAAAKNHBAAAA Successful, able hearts cite then contents. Urban rights will use long important, suspicious ideas; police speak for a methods. Plans seek no longer good gardens Sports sailing 4.39 8675.35 2.91608873856334289 +AAAAAAAAKNNBAAAA Scientific packages make banks. Then important parents must get front, little bact Sports sailing 4.23 6135.42 2.06232937787597103 +AAAAAAAALGNBAAAA Also long ways should not give only now good resources. Previous, economic units s Sports sailing 4.65 389.74 0.13100525338662731 +AAAAAAAALPEBAAAA Social years attend. Bloody wee Sports sailing 1.94 3178.08 1.06826390845941533 +AAAAAAAAMFBCAAAA Capital, foreign problems Sports sailing 3.60 1277.78 0.42950657533834004 +AAAAAAAAMLMCAAAA Original, major nations should come once more now permanent feet. Prizes revise thus with the spots. Aside ordinary studies can learn l Sports sailing 1.46 7468.82 2.51053178169833686 +AAAAAAAAMOMDAAAA Industrial, open sites would throw before a men. Also p Sports sailing 7.20 1089.19 0.36611487642064095 +AAAAAAAANJDDAAAA Loose patients used to look at all companies. Old, low centres may illustr Sports sailing 6.35 7701.71 2.58881426094401766 +AAAAAAAAOGBEAAAA Especially moral students used to keep guilty, bizarre things. Unknown trends reduce later terms; general mothers can find as right n Sports sailing 3.35 12086.74 4.06277630296680815 +AAAAAAAAOIKCAAAA Origins would come sales. Educational eyes could invite actually stupid, forei Sports sailing 3.77 9292.44 3.12351428331716300 +AAAAAAAAOKFDAAAA Legal, secondary sales elect. Big years appeal low with a doubts. Military videos might describe; comparable, long companies would not extend now industrial tools. Even ne Sports sailing 5.45 1828.50 0.61462284039987695 +AAAAAAAAOPACAAAA Additional organisations will adopt usually schemes. Conventional problems should not create attacks. Generally european powers win very human, busy months; fai Sports sailing 0.87 6498.29 2.18430268391693540 +AAAAAAAAOPGCAAAA Wrong, local indians train excellent, comprehensive holidays. Meals c Sports sailing 60.65 1510.40 0.50769829813506926 +AAAAAAAAOPLAAAAA National shareholders learn. Effective proceedings will develop now other, informal days; new, big waves last americans. Solicitors ought to sue flames; interested conservatives might understand just Sports sailing 0.24 5784.43 1.94434935558887624 +AAAAAAAAPHAAAAAA Ambitious exceptions appoint. V Sports sailing 7.35 9044.55 3.04018977912972767 +AAAAAAAAPNIBAAAA Proceedings mi Sports sailing 7.11 4105.60 1.38003584005782598 +AAAAAAAAABOBAAAA Again standard families change literally. Narrow lips work certainly carefully vast stages. Drugs see also right factors. Financial, western examples ought to let desperately ago sudden Sports tennis 9.39 6556.29 1.81601129267527792 +AAAAAAAAACFDAAAA Late global concepts shall understand very quiet, upper heads. Already english buildings make women. Others try. Please minimal agreements conflict largely forthcoming police. Sports tennis 4.33 7426.08 2.05693237186122454 +AAAAAAAAACPDAAAA Seriously complete characteristics make forward in a projects. Industries should rise then also new departments. Physical babies encourage under to a workers. Personal, beautiful ministers cont Sports tennis 0.82 14172.38 3.92557408596710262 +AAAAAAAAAJOCAAAA Whole, new meetings may last; free plans broaden there mammals. Public, honest areas may risk on a profits. Good, normal generations ought to walk almost over a reductions. Otherwise basic s Sports tennis 4.88 8723.48 2.41629613568450044 +AAAAAAAAAKOCAAAA Economic, content activit Sports tennis 5.07 16087.57 4.45605804375706699 +AAAAAAAAAPDCAAAA Women would come fair unaware, current bars. Villages may go then on a neighbours. Early numbers should not change however cr Sports tennis 2.92 13912.86 3.85369025369685708 +AAAAAAAABAPAAAAA Women should leave also annual, marginal techniques; intellectual, appropriate factors could think profits. Neverthe Sports tennis 8.24 23633.13 6.54608489881669218 +AAAAAAAABFHAAAAA Of course equal nee Sports tennis 3.49 11949.65 3.30990534944566741 +AAAAAAAABJPAAAAA Free representatives can fall much prime, useful banks. Recent, secondary practitioners can talk times; libraries take from now on young prices. Bodies appear only yellow rates. Second Sports tennis 6.85 7304.83 2.02334762054045053 +AAAAAAAABMJDAAAA Costly offices collect officially for a debts; readers greet. Women get by a write Sports tennis 3.22 2864.47 0.79342278446035080 +AAAAAAAACKLDAAAA Rapidly main banks shall not bring extremely decades. For example main clothes might not see less. Certainly co Sports tennis 3.15 5004.38 1.38615140465694887 +AAAAAAAACNIDAAAA Just able pounds should join then successful modern pieces. Associated, sorry clubs pay close issues. Resources will e Sports tennis 7.67 7567.71 2.09616213128028617 +AAAAAAAADHGDAAAA Necessary times believe probably. Cruel traders know ho Sports tennis 92.95 7731.85 2.14162688247032202 +AAAAAAAADLEBAAAA Funny, armed savings go yet thin Sports tennis 3.97 3362.82 0.93145957473422897 +AAAAAAAADPICAAAA Elected, marvellous advisers may not pass all in a programmes. Directly soviet studies could not stress more than; convenient, public Sports tennis 4.67 18.70 0.00517966886349257 +AAAAAAAAEAGBAAAA Men could remove only; economic, clear children raise public, extensive poli Sports tennis 5.04 2721.49 0.75381909172761457 +AAAAAAAAECHBAAAA Able, common villages read. Only social grounds remember e Sports tennis 2.08 2677.23 0.74155961879188295 +AAAAAAAAEIEDAAAA Successful parties see once on a ideas. Scottish, natural men would not examine regulatory, multiple payments. Steadily loc Sports tennis 2.55 8031.03 2.22449604453340795 +AAAAAAAAEODCAAAA Current, Sports tennis 0.47 18310.05 5.07165753336856247 +AAAAAAAAFAODAAAA Years may speak to a Sports tennis 2.02 3056.11 0.84650469574375807 +AAAAAAAAFLGDAAAA Separate, comfortable consumers get. Tests work even high, different faces. Hars Sports tennis 8.09 11878.41 3.29017274998923903 +AAAAAAAAGIIBAAAA Much critical possibilities might ensure; hence northern ways may persuade much japanese, running notes. Small, ed Sports tennis 8.53 8171.42 2.26338233927916847 +AAAAAAAAGLPDAAAA As specific ears worry also labour components. Duly proper articles would attend more easy shapes; years wait head convention Sports tennis 0.85 11273.32 3.12257029904748936 +AAAAAAAAGMEBAAAA Early, experimental factors mean usually suitable detectives; just black assets must not store only. So british employers must see elaborate, complete pages. Mental years should t Sports tennis 88.56 15092.59 4.18046088194969605 +AAAAAAAAHNOAAAAA Social, substantial orders would not offset however to a colleagues. Small students give for sure husbands. Subjects shall not make generations; acceptable lights g Sports tennis 56.30 5682.58 1.57400442194147617 +AAAAAAAAIDKCAAAA I Sports tennis 1.04 4973.48 1.37759248658839698 +AAAAAAAAIFCBAAAA Automatic amounts may find more in a regulations. Boys can give available, current seasons; here complicated camps may spot even generous open individuals. Channels remain currently protest Sports tennis 8.43 3330.22 0.92242977767808685 +AAAAAAAAIHODAAAA Points used to find cool typical managers. However military horses understand indeed inc periods. Coloured developments could make very roots. Sports tennis 8.52 11481.61 3.18026405453288334 +AAAAAAAAIMKBAAAA Sides express even new women. Also joint markets should switch authorities. Trees would play always about a skills. Teams deprive future pubs; ways recover national, old days. Rea Sports tennis 90.25 3634.02 1.00657862263685918 +AAAAAAAAKADEAAAA Secret children will start in short familie Sports tennis 38.21 13612.04 3.77036683190456646 +AAAAAAAAKEBCAAAA Other, general countries keep successfully teachers. Major, traditional relationships could not become in a subjects. Constant observers wil Sports tennis 99.16 7979.51 2.21022564133302628 +AAAAAAAAKKMAAAAA Upper, industrial years shall opera Sports tennis 1.58 369.36 0.10230815462136981 +AAAAAAAALBGEAAAA Afraid, spanish matt Sports tennis 3.06 141.37 0.03915774263272431 +AAAAAAAALBKDAAAA Light, social animals resist instead then female societies. Also informal minutes shall not implement. Servants win. Hands will a Sports tennis 8.30 3341.21 0.92547387183903783 +AAAAAAAALGIAAAAA Modest, educational principles would Sports tennis 6.42 18707.39 5.18171580215038800 +AAAAAAAALHPBAAAA Far little eyes can happen pp.. Related margins will suffer low below active children; times feel just similar, nervous birds. Vegetabl Sports tennis 9.01 813.78 0.22540700148304722 +AAAAAAAALKHDAAAA Then various shoes date good, bad shops. Here open rats match badly well dual games. No doubt small kids answer much points. Completely free services shall understand. Following patients Sports tennis 5.46 1154.69 0.31983485775327459 +AAAAAAAALODAAAAA Widely free parties would find in a problems. Men like parties; straight a Sports tennis 8.95 10297.10 2.85216942536199653 +AAAAAAAAMDACAAAA Tired rights free. Paintings sell Sports tennis 8.06 5429.22 1.50382683353214583 +AAAAAAAANAAAAAAA Meetings improve early women. Even likely variables might want approxi Sports tennis 2.56 7342.79 2.03386207134570068 +AAAAAAAANKODAAAA Growing jews see only grey tactics. Also indian parts ought to provide pretty other, canadian ways. Countries shall correspond really to a doubts. Star sounds ought to mean further at a steps. Sports tennis 8.04 4423.03 1.22512464028307694 +AAAAAAAAOCCBAAAA Else single arrangements will not keep approximately from a teachers. Large levels tolerate daily financial, critical others. Properties make a Sports tennis 0.30 5475.18 1.51655718545546767 +AAAAAAAAOEBEAAAA Equivalent, important points would not reject foreign, high mountains. Always alive cups mark near the games. Sons will not stay extremely. Unfortunatel Sports tennis 0.19 5314.97 1.47218099568968454 +AAAAAAAAOEDBAAAA Confidential companies can write highly; potentially new children mix sympathetically military, economic gains. Various, traditional designers make in a measurements. Individuals tell only se Sports tennis 7.12 1837.86 0.50906450360740392 +AAAAAAAAONEBAAAA Examples show waves. Currently representative farmers should put like a customers. Both full rights practise with a police. Legal re Sports tennis 4.24 735.27 0.20366070188557120 +AAAAAAAAPCOAAAAA Part Sports tennis 6.53 4928.46 1.36512250304644856 +AAAAAAAAPGOCAAAA Great, big arts will not let brilliant pp.. Real, new or Sports tennis 0.88 13772.83 3.81490367450140978 +AAAAAAAAPHPDAAAA Inc presents cannot break often subjects. Of course capital services would pay. Systems cannot Sports tennis 9.67 3395.45 0.94049768141956387 +AAAAAAAAPJNAAAAA Parts may refuse primarily old holidays. Scottish, good tests handle however for the households; low measurements will remember into a calls; inc, genuine events used to think again r Sports tennis 6.88 733.87 0.20327291918990865 +AAAAAAAAPMOCAAAA Literary pai Sports tennis 2.68 3317.04 0.91877908058606374 +AAAAAAAAPOHBAAAA Themes would not reflect on the jeans. Traditional relations would not force mildly smal Sports tennis 9.89 1274.76 0.35309276365913303 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 177517236ef88..70e69843f743c 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 @@ -584,21 +584,6 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { } } - 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") @@ -633,6 +618,48 @@ trait CharVarcharTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("c ")) } } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - join keys") { + withTable("t1", "t2") { + sql(s"CREATE TABLE t1(v VARCHAR(3), c CHAR(5)) USING $format") + sql(s"CREATE TABLE t2(v VARCHAR(5), c CHAR(8)) USING $format") + sql("INSERT INTO t1 VALUES ('c', 'b')") + sql("INSERT INTO t2 VALUES ('a', 'b')") + Seq("t1.c = t2.c", "t2.c = t1.c", + "t1.c = 'b'", "'b' = t1.c", "t1.c = 'b '", "'b ' = t1.c", + "t1.c = 'b '", "'b ' = t1.c").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT v FROM t1 + |WHERE 'a' IN (SELECT v FROM t2 WHERE $predicate) + """.stripMargin), + Row("c")) + } + } + } + + test("SPARK-34833: right-padding applied correctly for correlated subqueries - other preds") { + withTable("t") { + sql(s"CREATE TABLE t(c0 INT, c1 CHAR(5), c2 CHAR(7)) USING $format") + sql("INSERT INTO t VALUES (1, 'abc', 'abc')") + Seq("c1 = 'abc'", "'abc' = c1", "c1 = 'abc '", "'abc ' = c1", + "c1 = 'abc '", "'abc ' = c1", "c1 = c2", "c2 = c1", + "c1 IN ('xxx', 'abc', 'xxxxx')", "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN ('xxx', 'abc ', 'xxxxx')", + "c1 IN (c2)", "c2 IN (c1)").foreach { predicate => + checkAnswer(sql( + s""" + |SELECT c0 FROM t t1 + |WHERE ( + | SELECT count(*) AS c + | FROM t + | WHERE c0 = t1.c0 AND $predicate + |) > 0 + """.stripMargin), + Row(1)) + } + } + } } // Some basic char/varchar tests which doesn't rely on table implementation. @@ -779,6 +806,18 @@ class FileSourceCharVarcharTestSuite extends CharVarcharTestSuite with SharedSpa } } + test("SPARK-35359: create table and insert data over length values") { + Seq("char", "varchar").foreach { typ => + withSQLConf((SQLConf.LEGACY_CHAR_VARCHAR_AS_STRING.key, "true")) { + withTable("t") { + sql(s"CREATE TABLE t (col $typ(2)) using $format") + sql("INSERT INTO t SELECT 'aaa'") + checkAnswer(sql("select * from t"), Row("aaa")) + } + } + } + } + test("alter table set location w/ fit length values") { Seq("char", "varchar").foreach { typ => withTempPath { dir => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index 01b1508d034c3..186091db19280 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -1660,6 +1660,61 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession { StructType(Seq(StructField("a", IntegerType, nullable = true)))) } + test("SPARK-35213: chained withField operations should have correct schema for new columns") { + val df = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("data", NullType)))) + + checkAnswer( + df.withColumn("data", struct() + .withField("a", struct()) + .withField("b", struct()) + .withField("a.aa", lit("aa1")) + .withField("b.ba", lit("ba1")) + .withField("a.ab", lit("ab1"))), + Row(Row(Row("aa1", "ab1"), Row("ba1"))) :: Nil, + StructType(Seq( + StructField("data", StructType(Seq( + StructField("a", StructType(Seq( + StructField("aa", StringType, nullable = false), + StructField("ab", StringType, nullable = false) + )), nullable = false), + StructField("b", StructType(Seq( + StructField("ba", StringType, nullable = false) + )), nullable = false) + )), nullable = false) + )) + ) + } + + test("SPARK-35213: optimized withField operations should maintain correct nested struct " + + "ordering") { + val df = spark.createDataFrame( + sparkContext.parallelize(Row(null) :: Nil), + StructType(Seq(StructField("data", NullType)))) + + checkAnswer( + df.withColumn("data", struct() + .withField("a", struct().withField("aa", lit("aa1"))) + .withField("b", struct().withField("ba", lit("ba1"))) + ) + .withColumn("data", col("data").withField("b.bb", lit("bb1"))) + .withColumn("data", col("data").withField("a.ab", lit("ab1"))), + Row(Row(Row("aa1", "ab1"), Row("ba1", "bb1"))) :: Nil, + StructType(Seq( + StructField("data", StructType(Seq( + StructField("a", StructType(Seq( + StructField("aa", StringType, nullable = false), + StructField("ab", StringType, nullable = false) + )), nullable = false), + StructField("b", StructType(Seq( + StructField("ba", StringType, nullable = false), + StructField("bb", StringType, nullable = false) + )), nullable = false) + )), nullable = false) + )) + ) + } test("dropFields should throw an exception if called on a non-StructType column") { intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index c6d134b16d06d..b5d0ebd307fca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -1091,6 +1091,25 @@ class DataFrameAggregateSuite extends QueryTest val df = spark.sql(query) checkAnswer(df, Row(0, "0", 0, 0) :: Row(-1, "1", 1, 1) :: Row(-2, "2", 2, 2) :: Nil) } + + test("SPARK-34713: group by CreateStruct with ExtractValue") { + val structDF = Seq(Tuple1(1 -> 1)).toDF("col") + checkAnswer(structDF.groupBy(struct($"col._1")).count().select("count"), Row(1)) + + val arrayOfStructDF = Seq(Tuple1(Seq(1 -> 1))).toDF("col") + checkAnswer(arrayOfStructDF.groupBy(struct($"col._1")).count().select("count"), Row(1)) + + val mapDF = Seq(Tuple1(Map("a" -> "a"))).toDF("col") + checkAnswer(mapDF.groupBy(struct($"col.a")).count().select("count"), Row(1)) + + val nonStringMapDF = Seq(Tuple1(Map(1 -> 1))).toDF("col") + // Spark implicit casts string literal "a" to int to match the key type. + checkAnswer(nonStringMapDF.groupBy(struct($"col.a")).count().select("count"), Row(1)) + + val arrayDF = Seq(Tuple1(Seq(1))).toDF("col") + val e = intercept[AnalysisException](arrayDF.groupBy(struct($"col.a")).count()) + assert(e.message.contains("requires integral type")) + } } case class B(c: Option[Double]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index aa1678e4f82fa..9baad7c026693 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -3629,6 +3629,29 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession { df.select(map(map_entries($"m"), lit(1))), Row(Map(Seq(Row(1, "a")) -> 1))) } + + test("SPARK-34794: lambda variable name issues in nested functions") { + val df1 = Seq((Seq(1, 2), Seq("a", "b"))).toDF("numbers", "letters") + + checkAnswer(df1.select(flatten(transform($"numbers", (number: Column) => + transform($"letters", (letter: Column) => + struct(number, letter))))), + Seq(Row(Seq(Row(1, "a"), Row(1, "b"), Row(2, "a"), Row(2, "b")))) + ) + checkAnswer(df1.select(flatten(transform($"numbers", (number: Column, i: Column) => + transform($"letters", (letter: Column, j: Column) => + struct(number + j, concat(letter, i)))))), + Seq(Row(Seq(Row(1, "a0"), Row(2, "b0"), Row(2, "a1"), Row(3, "b1")))) + ) + + val df2 = Seq((Map("a" -> 1, "b" -> 2), Map("a" -> 2, "b" -> 3))).toDF("m1", "m2") + + checkAnswer(df2.select(map_zip_with($"m1", $"m2", (k1: Column, ov1: Column, ov2: Column) => + map_zip_with($"m1", $"m2", (k2: Column, iv1: Column, iv2: Column) => + ov1 + iv1 + ov2 + iv2))), + Seq(Row(Map("a" -> Map("a" -> 6, "b" -> 8), "b" -> Map("a" -> 8, "b" -> 10)))) + ) + } } object DataFrameFunctionsSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 091877f7cac37..23c2349f89574 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -460,4 +460,29 @@ class DataFrameNaFunctionsSuite extends QueryTest with SharedSparkSession { Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Row(0, 0L, 0.toShort, 0.toByte, Float.NaN, Double.NaN) :: Nil) } + + test("SPARK-34417 - test fillMap() for column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col") + .na.fill(Map("`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for qualified-column with a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("ColWith.Dot", "Col").as("testDF") + .na.fill(Map("testDF.`ColWith.Dot`" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } + + test("SPARK-34417 - test fillMap() for column without a dot in the name" + + " and dataframe with another column having a dot in the name") { + val na = "n/a" + checkAnswer( + Seq(("abc", 23L), ("def", 44L), (null, 0L)).toDF("Col", "ColWith.Dot") + .na.fill(Map("Col" -> na)), + Row("abc", 23) :: Row("def", 44L) :: Row(na, 0L) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 8d95f83e49f3a..61a528c9a1b7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCod import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} @@ -2584,6 +2585,105 @@ class DataFrameSuite extends QueryTest val col2 = df.colRegex("test\n_table.`tes.*\n.*mn`") checkAnswer(df.select(col2), Row(1) :: Row(2) :: Row(3) :: Nil) } + + test("SPARK-34763: col(), $\"\", df(\"name\") should handle quoted column name properly") { + val df1 = spark.sql("SELECT 'col1' AS `a``b.c`") + checkAnswer(df1.selectExpr("`a``b.c`"), Row("col1")) + checkAnswer(df1.select(df1("`a``b.c`")), Row("col1")) + checkAnswer(df1.select(col("`a``b.c`")), Row("col1")) + checkAnswer(df1.select($"`a``b.c`"), Row("col1")) + + val df2 = df1.as("d.e`f") + checkAnswer(df2.selectExpr("`a``b.c`"), Row("col1")) + checkAnswer(df2.select(df2("`a``b.c`")), Row("col1")) + checkAnswer(df2.select(col("`a``b.c`")), Row("col1")) + checkAnswer(df2.select($"`a``b.c`"), Row("col1")) + + checkAnswer(df2.selectExpr("`d.e``f`.`a``b.c`"), Row("col1")) + checkAnswer(df2.select(df2("`d.e``f`.`a``b.c`")), Row("col1")) + checkAnswer(df2.select(col("`d.e``f`.`a``b.c`")), Row("col1")) + checkAnswer(df2.select($"`d.e``f`.`a``b.c`"), Row("col1")) + + val df3 = df1.as("*-#&% ?") + checkAnswer(df3.selectExpr("`*-#&% ?`.`a``b.c`"), Row("col1")) + checkAnswer(df3.select(df3("*-#&% ?.`a``b.c`")), Row("col1")) + checkAnswer(df3.select(col("*-#&% ?.`a``b.c`")), Row("col1")) + checkAnswer(df3.select($"*-#&% ?.`a``b.c`"), Row("col1")) + } + + test("SPARK-34776: Nested column pruning should not prune Window produced attributes") { + val df = Seq( + ("t1", "123", "bob"), + ("t1", "456", "bob"), + ("t2", "123", "sam") + ).toDF("type", "value", "name") + + val test = df.select( + $"*", + struct(count($"*").over(Window.partitionBy($"type", $"value", $"name")) + .as("count"), $"name").as("name_count") + ).select( + $"*", + max($"name_count").over(Window.partitionBy($"type", $"value")).as("best_name") + ) + checkAnswer(test.select($"best_name.name"), Row("bob") :: Row("bob") :: Row("sam") :: Nil) + } + + test("SPARK-34829: Multiple applications of typed ScalaUDFs in higher order functions work") { + val reverse = udf((s: String) => s.reverse) + val reverse2 = udf((b: Bar2) => Bar2(b.s.reverse)) + + val df = Seq(Array("abc", "def")).toDF("array") + val test = df.select(transform(col("array"), s => reverse(s))) + checkAnswer(test, Row(Array("cba", "fed")) :: Nil) + + val df2 = Seq(Array(Bar2("abc"), Bar2("def"))).toDF("array") + val test2 = df2.select(transform(col("array"), b => reverse2(b))) + checkAnswer(test2, Row(Array(Row("cba"), Row("fed"))) :: Nil) + + val df3 = Seq(Map("abc" -> 1, "def" -> 2)).toDF("map") + val test3 = df3.select(transform_keys(col("map"), (s, _) => reverse(s))) + checkAnswer(test3, Row(Map("cba" -> 1, "fed" -> 2)) :: Nil) + + val df4 = Seq(Map(Bar2("abc") -> 1, Bar2("def") -> 2)).toDF("map") + val test4 = df4.select(transform_keys(col("map"), (b, _) => reverse2(b))) + checkAnswer(test4, Row(Map(Row("cba") -> 1, Row("fed") -> 2)) :: Nil) + + val df5 = Seq(Map(1 -> "abc", 2 -> "def")).toDF("map") + val test5 = df5.select(transform_values(col("map"), (_, s) => reverse(s))) + checkAnswer(test5, Row(Map(1 -> "cba", 2 -> "fed")) :: Nil) + + val df6 = Seq(Map(1 -> Bar2("abc"), 2 -> Bar2("def"))).toDF("map") + val test6 = df6.select(transform_values(col("map"), (_, b) => reverse2(b))) + checkAnswer(test6, Row(Map(1 -> Row("cba"), 2 -> Row("fed"))) :: Nil) + + val reverseThenConcat = udf((s1: String, s2: String) => s1.reverse ++ s2.reverse) + val reverseThenConcat2 = udf((b1: Bar2, b2: Bar2) => Bar2(b1.s.reverse ++ b2.s.reverse)) + + val df7 = Seq((Map(1 -> "abc", 2 -> "def"), Map(1 -> "ghi", 2 -> "jkl"))).toDF("map1", "map2") + val test7 = + df7.select(map_zip_with(col("map1"), col("map2"), (_, s1, s2) => reverseThenConcat(s1, s2))) + checkAnswer(test7, Row(Map(1 -> "cbaihg", 2 -> "fedlkj")) :: Nil) + + val df8 = Seq((Map(1 -> Bar2("abc"), 2 -> Bar2("def")), + Map(1 -> Bar2("ghi"), 2 -> Bar2("jkl")))).toDF("map1", "map2") + val test8 = + df8.select(map_zip_with(col("map1"), col("map2"), (_, b1, b2) => reverseThenConcat2(b1, b2))) + checkAnswer(test8, Row(Map(1 -> Row("cbaihg"), 2 -> Row("fedlkj"))) :: Nil) + + val df9 = Seq((Array("abc", "def"), Array("ghi", "jkl"))).toDF("array1", "array2") + val test9 = + df9.select(zip_with(col("array1"), col("array2"), (s1, s2) => reverseThenConcat(s1, s2))) + checkAnswer(test9, Row(Array("cbaihg", "fedlkj")) :: Nil) + + val df10 = Seq((Array(Bar2("abc"), Bar2("def")), Array(Bar2("ghi"), Bar2("jkl")))) + .toDF("array1", "array2") + val test10 = + df10.select(zip_with(col("array1"), col("array2"), (b1, b2) => reverseThenConcat2(b1, b2))) + checkAnswer(test10, Row(Array(Row("cbaihg"), Row("fedlkj"))) :: Nil) + } } case class GroupByKey(a: Int, b: Int) + +case class Bar2(s: String) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala index e47a6a68a0a9c..955d0f9193e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala @@ -28,7 +28,9 @@ import org.apache.spark.sql.types.StringType * Benchmark for Dataset typed operations comparing with DataFrame and RDD versions. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/DatasetBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala index 55437aaa47298..db7b0dd4b67e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DynamicPartitionPruningSuite.scala @@ -1361,6 +1361,26 @@ abstract class DynamicPartitionPruningSuiteBase checkAnswer(df, Nil) } } + + test("SPARK-34436: DPP support LIKE ANY/ALL expression") { + withSQLConf(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED.key -> "true") { + val df = sql( + """ + |SELECT date_id, product_id FROM fact_sk f + |JOIN dim_store s + |ON f.store_id = s.store_id WHERE s.country LIKE ANY ('%D%E%', '%A%B%') + """.stripMargin) + + checkPartitionPruningPredicate(df, false, true) + + checkAnswer(df, + Row(1030, 2) :: + Row(1040, 2) :: + Row(1050, 2) :: + Row(1060, 2) :: Nil + ) + } + } } class DynamicPartitionPruningSuiteAEOff extends DynamicPartitionPruningSuiteBase { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala index fd1706a7e6f47..c18f4782a28a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ExplainSuite.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} +import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.TestOptionsSource import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, StructField, StructType} @@ -69,6 +71,18 @@ trait ExplainSuiteHelper extends QueryTest with SharedSparkSession { protected def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { checkKeywordsExistsInExplain(df, ExtendedMode, keywords: _*) } + + /** + * Runs the plan and makes sure the plans does not contain any of the keywords. + */ + protected def checkKeywordsNotExistsInExplain( + df: DataFrame, mode: ExplainMode, keywords: String*): Unit = { + withNormalizedExplain(df, mode) { normalizedOutput => + for (key <- keywords) { + assert(!normalizedOutput.contains(key)) + } + } + } } class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite { @@ -347,6 +361,45 @@ class ExplainSuite extends ExplainSuiteHelper with DisableAdaptiveExecutionSuite Nil: _*) } + test("SPARK-34970: Redact Map type options in explain output") { + val password = "MyPassWord" + val token = "MyToken" + val value = "value" + val options = Map("password" -> password, "token" -> token, "key" -> value) + val cmd = SaveIntoDataSourceCommand(spark.range(10).logicalPlan, new TestOptionsSource, + options, SaveMode.Overwrite) + + Seq(SimpleMode, ExtendedMode, FormattedMode).foreach { mode => + checkKeywordsExistsInExplain(cmd, mode, value) + } + Seq(SimpleMode, ExtendedMode, CodegenMode, CostMode, FormattedMode).foreach { mode => + checkKeywordsNotExistsInExplain(cmd, mode, password) + checkKeywordsNotExistsInExplain(cmd, mode, token) + } + } + + test("SPARK-34970: Redact CaseInsensitiveMap type options in explain output") { + val password = "MyPassWord" + val token = "MyToken" + val value = "value" + val tableName = "t" + withTable(tableName) { + val df1 = spark.range(10).toDF() + df1.write.format("json").saveAsTable(tableName) + val df2 = spark.read + .option("key", value) + .option("password", password) + .option("token", token) + .table(tableName) + + checkKeywordsExistsInExplain(df2, ExtendedMode, value) + Seq(SimpleMode, ExtendedMode, CodegenMode, CostMode, FormattedMode).foreach { mode => + checkKeywordsNotExistsInExplain(df2, mode, password) + checkKeywordsNotExistsInExplain(df2, mode, token) + } + } + } + test("Dataset.toExplainString has mode as string") { val df = spark.range(10).toDF def assertExplainOutput(mode: ExplainMode): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala new file mode 100644 index 0000000000000..4e7fe8455ff93 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileScanSuite.scala @@ -0,0 +1,374 @@ +/* + * 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 + +import scala.collection.mutable + +import com.google.common.collect.ImmutableMap +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{And, Expression, IsNull, LessThan} +import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitionSpec} +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan +import org.apache.spark.sql.execution.datasources.v2.json.JsonScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.datasources.v2.text.TextScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +trait FileScanSuiteBase extends SharedSparkSession { + private def newPartitioningAwareFileIndex() = { + new PartitioningAwareFileIndex(spark, Map.empty, None) { + override def partitionSpec(): PartitionSpec = { + PartitionSpec.emptySpec + } + + override protected def leafFiles: mutable.LinkedHashMap[Path, FileStatus] = { + mutable.LinkedHashMap.empty + } + + override protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] = { + Map.empty + } + + override def rootPaths: Seq[Path] = { + Seq.empty + } + + override def refresh(): Unit = {} + } + } + + type ScanBuilder = ( + SparkSession, + PartitioningAwareFileIndex, + StructType, + StructType, + StructType, + Array[Filter], + CaseInsensitiveStringMap, + Seq[Expression], + Seq[Expression]) => FileScan + + def run(scanBuilders: Seq[(String, ScanBuilder, Seq[String])]): Unit = { + val dataSchema = StructType.fromDDL("data INT, partition INT, other INT") + val dataSchemaNotEqual = StructType.fromDDL("data INT, partition INT, other INT, new INT") + val readDataSchema = StructType.fromDDL("data INT") + val readDataSchemaNotEqual = StructType.fromDDL("data INT, other INT") + val readPartitionSchema = StructType.fromDDL("partition INT") + val readPartitionSchemaNotEqual = StructType.fromDDL("partition INT, other INT") + val pushedFilters = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 0))) + val pushedFiltersNotEqual = + Array[Filter](sources.And(sources.IsNull("data"), sources.LessThan("data", 1))) + val optionsMap = ImmutableMap.of("key", "value") + val options = new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)) + val optionsNotEqual = + new CaseInsensitiveStringMap(ImmutableMap.copyOf(ImmutableMap.of("key2", "value2"))) + val partitionFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val partitionFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + val dataFilters = Seq(And(IsNull('data.int), LessThan('data.int, 0))) + val dataFiltersNotEqual = Seq(And(IsNull('data.int), LessThan('data.int, 1))) + + scanBuilders.foreach { case (name, scanBuilder, exclusions) => + test(s"SPARK-33482: Test $name equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanEquals = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema.copy(), + readDataSchema.copy(), + readPartitionSchema.copy(), + pushedFilters.clone(), + new CaseInsensitiveStringMap(ImmutableMap.copyOf(optionsMap)), + Seq(partitionFilters: _*), + Seq(dataFilters: _*)) + + assert(scan === scanEquals) + } + + test(s"SPARK-33482: Test $name fileIndex not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val partitioningAwareFileIndexNotEqual = newPartitioningAwareFileIndex() + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndexNotEqual, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("dataSchema")) { + test(s"SPARK-33482: Test $name dataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchemaNotEqual, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name readDataSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchemaNotEqual, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name readPartitionSchema not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchemaNotEqual, + pushedFilters, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + if (!exclusions.contains("pushedFilters")) { + test(s"SPARK-33482: Test $name pushedFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFiltersNotEqual, + options, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + } + + test(s"SPARK-33482: Test $name options not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + optionsNotEqual, + partitionFilters, + dataFilters) + + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name partitionFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFiltersNotEqual, + dataFilters) + assert(scan !== scanNotEqual) + } + + test(s"SPARK-33482: Test $name dataFilters not equals") { + val partitioningAwareFileIndex = newPartitioningAwareFileIndex() + + val scan = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFilters) + + val scanNotEqual = scanBuilder( + spark, + partitioningAwareFileIndex, + dataSchema, + readDataSchema, + readPartitionSchema, + pushedFilters, + options, + partitionFilters, + dataFiltersNotEqual) + assert(scan !== scanNotEqual) + } + } + } +} + +class FileScanSuite extends FileScanSuiteBase { + val scanBuilders = Seq[(String, ScanBuilder, Seq[String])]( + ("ParquetScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + ParquetScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, f, o, pf, df), + Seq.empty), + ("OrcScan", + (s, fi, ds, rds, rps, f, o, pf, df) => + OrcScan(s, s.sessionState.newHadoopConf(), fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("CSVScan", + (s, fi, ds, rds, rps, f, o, pf, df) => CSVScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("JsonScan", + (s, fi, ds, rds, rps, f, o, pf, df) => JsonScan(s, fi, ds, rds, rps, o, f, pf, df), + Seq.empty), + ("TextScan", + (s, fi, _, rds, rps, _, o, pf, df) => TextScan(s, fi, rds, rps, o, pf, df), + Seq("dataSchema", "pushedFilters"))) + + run(scanBuilders) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala b/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala new file mode 100644 index 0000000000000..efbe3f53828be --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/GenTPCDSData.scala @@ -0,0 +1,395 @@ +/* + * 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 + +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.immutable.Stream +import scala.sys.process._ +import scala.util.Try + +import org.apache.spark.SparkContext +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.functions.{col, rpad} +import org.apache.spark.sql.types.{CharType, StringType, StructField, StructType, VarcharType} + +// The classes in this file are basically moved from https://github.com/databricks/spark-sql-perf + +/** + * Using ProcessBuilder.lineStream produces a stream, that uses + * a LinkedBlockingQueue with a default capacity of Integer.MAX_VALUE. + * + * This causes OOM if the consumer cannot keep up with the producer. + * + * See scala.sys.process.ProcessBuilderImpl.lineStream + */ +object BlockingLineStream { + + // See scala.sys.process.Streamed + private final class BlockingStreamed[T]( + val process: T => Unit, + val done: Int => Unit, + val stream: () => Stream[T]) + + // See scala.sys.process.Streamed + private object BlockingStreamed { + // scala.process.sys.Streamed uses default of Integer.MAX_VALUE, + // which causes OOMs if the consumer cannot keep up with producer. + val maxQueueSize = 65536 + + def apply[T](nonzeroException: Boolean): BlockingStreamed[T] = { + val q = new LinkedBlockingQueue[Either[Int, T]](maxQueueSize) + + def next(): Stream[T] = q.take match { + case Left(0) => Stream.empty + case Left(code) => + if (nonzeroException) scala.sys.error("Nonzero exit code: " + code) else Stream.empty + case Right(s) => Stream.cons(s, next()) + } + + new BlockingStreamed((s: T) => q put Right(s), code => q put Left(code), () => next()) + } + } + + // See scala.sys.process.ProcessImpl.Spawn + private object Spawn { + def apply(f: => Unit): Thread = apply(f, daemon = false) + def apply(f: => Unit, daemon: Boolean): Thread = { + val thread = new Thread() { override def run() = { f } } + thread.setDaemon(daemon) + thread.start() + thread + } + } + + def apply(command: Seq[String]): Stream[String] = { + val streamed = BlockingStreamed[String](true) + val process = command.run(BasicIO(false, streamed.process, None)) + Spawn(streamed.done(process.exitValue())) + streamed.stream() + } +} + +class Dsdgen(dsdgenDir: String) extends Serializable { + private val dsdgen = s"$dsdgenDir/dsdgen" + + def generate( + sparkContext: SparkContext, + tableName: String, + partitions: Int, + scaleFactor: Int): RDD[String] = { + val generatedData = { + sparkContext.parallelize(1 to partitions, partitions).flatMap { i => + val localToolsDir = if (new java.io.File(dsdgen).exists) { + dsdgenDir + } else if (new java.io.File(s"/$dsdgen").exists) { + s"/$dsdgenDir" + } else { + throw new IllegalStateException( + s"Could not find dsdgen at $dsdgen or /$dsdgen. Run install") + } + + // NOTE: RNGSEED is the RNG seed used by the data generator. Right now, it is fixed to 100. + val parallel = if (partitions > 1) s"-parallel $partitions -child $i" else "" + val commands = Seq( + "bash", "-c", + s"cd $localToolsDir && ./dsdgen -table $tableName -filter Y -scale $scaleFactor " + + s"-RNGSEED 100 $parallel") + BlockingLineStream(commands) + } + } + + generatedData.setName(s"$tableName, sf=$scaleFactor, strings") + generatedData + } +} + +class TPCDSTables(sqlContext: SQLContext, dsdgenDir: String, scaleFactor: Int) + extends TPCDSSchema with Logging with Serializable { + + private val dataGenerator = new Dsdgen(dsdgenDir) + + private def tables: Seq[Table] = tableColumns.map { case (tableName, schemaString) => + Table(tableName, StructType.fromDDL(schemaString)) + }.toSeq + + private case class Table(name: String, schema: StructType) { + private def df(numPartition: Int) = { + val generatedData = dataGenerator.generate( + sqlContext.sparkContext, name, numPartition, scaleFactor) + val rows = generatedData.mapPartitions { iter => + iter.map { l => + val values = l.split("\\|", -1).dropRight(1).map { v => + if (v.equals("")) { + // If the string value is an empty string, we turn it to a null + null + } else { + v + } + } + Row.fromSeq(values) + } + } + + val stringData = + sqlContext.createDataFrame( + rows, + StructType(schema.fields.map(f => StructField(f.name, StringType)))) + + val convertedData = { + val columns = schema.fields.map { f => + val c = f.dataType match { + // Needs right-padding for char types + case CharType(n) => rpad(Column(f.name), n, " ") + // Don't need a cast for varchar types + case _: VarcharType => col(f.name) + case _ => col(f.name).cast(f.dataType) + } + c.as(f.name) + } + stringData.select(columns: _*) + } + + convertedData + } + + def genData( + location: String, + format: String, + overwrite: Boolean, + clusterByPartitionColumns: Boolean, + filterOutNullPartitionValues: Boolean, + numPartitions: Int): Unit = { + val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Ignore + + val data = df(numPartitions) + val tempTableName = s"${name}_text" + data.createOrReplaceTempView(tempTableName) + + val writer = { + // treat non-partitioned tables as "one partition" that we want to coalesce + if (clusterByPartitionColumns) { + // in case data has more than maxRecordsPerFile, split into multiple writers to improve + // datagen speed files will be truncated to maxRecordsPerFile value, so the final + // result will be the same. + val numRows = data.count + val maxRecordPerFile = Try { + sqlContext.getConf("spark.sql.files.maxRecordsPerFile").toInt + }.getOrElse(0) + + if (maxRecordPerFile > 0 && numRows > maxRecordPerFile) { + val numFiles = (numRows.toDouble/maxRecordPerFile).ceil.toInt + logInfo(s"Coalescing into $numFiles files") + data.coalesce(numFiles).write + } else { + data.coalesce(1).write + } + } else { + data.write + } + } + writer.format(format).mode(mode) + logInfo(s"Generating table $name in database to $location with save mode $mode.") + writer.save(location) + sqlContext.dropTempTable(tempTableName) + } + } + + def genData( + location: String, + format: String, + overwrite: Boolean, + clusterByPartitionColumns: Boolean, + filterOutNullPartitionValues: Boolean, + tableFilter: String = "", + numPartitions: Int = 100): Unit = { + var tablesToBeGenerated = tables + + if (!tableFilter.isEmpty) { + tablesToBeGenerated = tablesToBeGenerated.filter(_.name == tableFilter) + if (tablesToBeGenerated.isEmpty) { + throw new RuntimeException("Bad table name filter: " + tableFilter) + } + } + + tablesToBeGenerated.foreach { table => + val tableLocation = s"$location/${table.name}" + table.genData(tableLocation, format, overwrite, clusterByPartitionColumns, + filterOutNullPartitionValues, numPartitions) + } + } +} + +class GenTPCDSDataConfig(args: Array[String]) { + var master: String = "local[*]" + var dsdgenDir: String = null + var location: String = null + var scaleFactor: Int = 1 + var format: String = "parquet" + var overwrite: Boolean = false + var clusterByPartitionColumns: Boolean = false + var filterOutNullPartitionValues: Boolean = false + var tableFilter: String = "" + var numPartitions: Int = 100 + + parseArgs(args.toList) + + private def parseArgs(inputArgs: List[String]): Unit = { + var args = inputArgs + + while (args.nonEmpty) { + args match { + case "--master" :: value :: tail => + master = value + args = tail + + case "--dsdgenDir" :: value :: tail => + dsdgenDir = value + args = tail + + case "--location" :: value :: tail => + location = value + args = tail + + case "--scaleFactor" :: value :: tail => + scaleFactor = toPositiveIntValue("Scale factor", value) + args = tail + + case "--format" :: value :: tail => + format = value + args = tail + + case "--overwrite" :: tail => + overwrite = true + args = tail + + case "--clusterByPartitionColumns" :: tail => + clusterByPartitionColumns = true + args = tail + + case "--filterOutNullPartitionValues" :: tail => + filterOutNullPartitionValues = true + args = tail + + case "--tableFilter" :: value :: tail => + tableFilter = value + args = tail + + case "--numPartitions" :: value :: tail => + numPartitions = toPositiveIntValue("Number of partitions", value) + args = tail + + case "--help" :: tail => + printUsageAndExit(0) + + case _ => + // scalastyle:off println + System.err.println("Unknown/unsupported param " + args) + // scalastyle:on println + printUsageAndExit(1) + } + } + + checkRequiredArguments() + } + + private def printUsageAndExit(exitCode: Int): Unit = { + // scalastyle:off + System.err.println(""" + |build/sbt "test:runMain [Options]" + |Options: + | --master the Spark master to use, default to local[*] + | --dsdgenDir location of dsdgen + | --location root directory of location to generate data in + | --scaleFactor size of the dataset to generate (in GB) + | --format generated data format, Parquet, ORC ... + | --overwrite whether to overwrite the data that is already there + | --clusterByPartitionColumns whether to shuffle to get partitions coalesced into single files + | --filterOutNullPartitionValues whether to filter out the partition with NULL key value + | --tableFilter comma-separated list of table names to generate (e.g., store_sales,store_returns), + | all the tables are generated by default + | --numPartitions how many dsdgen partitions to run - number of input tasks + """.stripMargin) + // scalastyle:on + System.exit(exitCode) + } + + private def toPositiveIntValue(name: String, v: String): Int = { + if (Try(v.toInt).getOrElse(-1) <= 0) { + // scalastyle:off println + System.err.println(s"$name must be a positive number") + // scalastyle:on println + printUsageAndExit(-1) + } + v.toInt + } + + private def checkRequiredArguments(): Unit = { + if (dsdgenDir == null) { + // scalastyle:off println + System.err.println("Must specify a dsdgen path") + // scalastyle:on println + printUsageAndExit(-1) + } + if (location == null) { + // scalastyle:off println + System.err.println("Must specify an output location") + // scalastyle:on println + printUsageAndExit(-1) + } + } +} + +/** + * This class generates TPCDS table data by using tpcds-kit: + * - https://github.com/databricks/tpcds-kit + * + * To run this: + * {{{ + * build/sbt "sql/test:runMain --dsdgenDir --location --scaleFactor 1" + * }}} + */ +object GenTPCDSData { + + def main(args: Array[String]): Unit = { + val config = new GenTPCDSDataConfig(args) + + val spark = SparkSession + .builder() + .appName(getClass.getName) + .master(config.master) + .getOrCreate() + + val tables = new TPCDSTables( + spark.sqlContext, + dsdgenDir = config.dsdgenDir, + scaleFactor = config.scaleFactor) + + tables.genData( + location = config.location, + format = config.format, + overwrite = config.overwrite, + clusterByPartitionColumns = config.clusterByPartitionColumns, + filterOutNullPartitionValues = config.filterOutNullPartitionValues, + tableFilter = config.tableFilter, + numPartitions = config.numPartitions) + + spark.stop() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index c7446c7a9f443..a0943437bc8b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} /** - * The base trait for DML - insert syntax + * The base trait for SQL INSERT. */ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { @@ -208,6 +208,55 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { checkAnswer(spark.table("t"), Row("1", null)) } } + + test("SPARK-34556: " + + "checking duplicate static partition columns should respect case sensitive conf") { + withTable("t") { + sql(s"CREATE TABLE t(i STRING, c string) USING PARQUET PARTITIONED BY (c)") + val e = intercept[AnalysisException] { + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + } + assert(e.getMessage.contains("Found duplicate keys 'c'")) + } + // The following code is skipped for Hive because columns stored in Hive Metastore is always + // case insensitive and we cannot create such table in Hive Metastore. + if (!format.startsWith("hive")) { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTable("t") { + sql(s"CREATE TABLE t(i int, c string, C string) USING PARQUET PARTITIONED BY (c, C)") + sql("INSERT OVERWRITE t PARTITION (c='2', C='3') VALUES (1)") + checkAnswer(spark.table("t"), Row(1, "2", "3")) + } + } + } + } + + test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { + val testingPolicies = if (format == "foo") { + // DS v2 doesn't support the legacy policy + Seq(SQLConf.StoreAssignmentPolicy.ANSI, SQLConf.StoreAssignmentPolicy.STRICT) + } else { + SQLConf.StoreAssignmentPolicy.values + } + testingPolicies.foreach { policy => + withSQLConf( + SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { + withTable("t") { + sql("create table t(a int, b string) using parquet partitioned by (a)") + policy match { + case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => + val errorMsg = intercept[NumberFormatException] { + sql("insert into t partition(a='ansi') values('ansi')") + }.getMessage + assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) + case SQLConf.StoreAssignmentPolicy.LEGACY => + sql("insert into t partition(a='ansi') values('ansi')") + checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) + } + } + } + } + } } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { 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 e42fb963d5f8c..6df57b1918d2d 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 @@ -33,10 +33,10 @@ import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.command.FunctionsCommand -import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -3596,6 +3596,9 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark Row("A", null, 5) :: Row("A", "{\"i\": 1}", 3) :: Row("A", "{\"i\": 2}", 2) :: Row("B", null, 1) :: Row("B", "{\"i\": 1}", 1) :: Row("C", null, 3) :: Row("C", "{\"i\": 1}", 3) :: Nil) + + assert(spark.table("t").groupBy($"c.json_string").count().schema.fieldNames === + Seq("json_string", "count")) } } @@ -3786,69 +3789,6 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } - test("SPARK-34212 Parquet should read decimals correctly") { - def readParquet(schema: String, path: File): DataFrame = { - spark.read.schema(schema).parquet(path.toString) - } - - withTempPath { path => - // a is int-decimal (4 bytes), b is long-decimal (8 bytes), c is binary-decimal (16 bytes) - val df = sql("SELECT 1.0 a, CAST(1.23 AS DECIMAL(17, 2)) b, CAST(1.23 AS DECIMAL(36, 2)) c") - df.write.parquet(path.toString) - - Seq(true, false).foreach { vectorizedReader => - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorizedReader.toString) { - // We can read the decimal parquet field with a larger precision, if scale is the same. - val schema = "a DECIMAL(9, 1), b DECIMAL(18, 2), c DECIMAL(38, 2)" - checkAnswer(readParquet(schema, path), df) - } - } - - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" - checkAnswer(readParquet(schema1, path), df) - val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" - checkAnswer(readParquet(schema2, path), Row(1, 1.2, 1.2)) - } - - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - Seq("a DECIMAL(3, 2)", "b DECIMAL(18, 1)", "c DECIMAL(37, 1)").foreach { schema => - val e = intercept[SparkException] { - readParquet(schema, path).collect() - }.getCause.getCause - assert(e.isInstanceOf[SchemaColumnConvertNotSupportedException]) - } - } - } - - // tests for parquet types without decimal metadata. - withTempPath { path => - val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") - df.write.parquet(path.toString) - - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) - checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) - checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) - checkAnswer(readParquet("c DECIMAL(11, 1)", path), Row(null)) - checkAnswer(readParquet("c DECIMAL(13, 0)", path), df.select("c")) - val e = intercept[SparkException] { - readParquet("d DECIMAL(3, 2)", path).collect() - }.getCause - assert(e.getMessage.contains("Please read this column/field as Spark BINARY type")) - } - - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - Seq("a DECIMAL(3, 2)", "c DECIMAL(18, 1)", "d DECIMAL(37, 1)").foreach { schema => - val e = intercept[SparkException] { - readParquet(schema, path).collect() - }.getCause.getCause - assert(e.isInstanceOf[SchemaColumnConvertNotSupportedException]) - } - } - } - } - test("SPARK-34421: Resolve temporary objects in temporary views with CTEs") { val tempFuncName = "temp_func" withUserDefinedFunction(tempFuncName -> true) { @@ -3926,6 +3866,48 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-34796: Avoid code-gen compilation error for LIMIT query") { + withTable("left_table", "empty_right_table", "output_table") { + spark.range(5).toDF("k").write.saveAsTable("left_table") + spark.range(0).toDF("k").write.saveAsTable("empty_right_table") + + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + spark.sql("CREATE TABLE output_table (k INT) USING parquet") + spark.sql( + """ + |INSERT INTO TABLE output_table + |SELECT t1.k FROM left_table t1 + |JOIN empty_right_table t2 + |ON t1.k = t2.k + |LIMIT 3 + """.stripMargin) + } + } + } + + test("SPARK-33482: Fix FileScan canonicalization") { + withSQLConf(SQLConf.USE_V1_SOURCE_LIST.key -> "") { + withTempPath { path => + spark.range(5).toDF().write.mode("overwrite").parquet(path.toString) + withTempView("t") { + spark.read.parquet(path.toString).createOrReplaceTempView("t") + val df = sql( + """ + |SELECT * + |FROM t AS t1 + |JOIN t AS t2 ON t2.id = t1.id + |JOIN t AS t3 ON t3.id = t2.id + |""".stripMargin) + df.collect() + val reusedExchanges = collect(df.queryExecution.executedPlan) { + case r: ReusedExchangeExec => r + } + assert(reusedExchanges.size == 1) + } + } + } + } } case class Foo(bar: Option[String]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala new file mode 100644 index 0000000000000..d309065701395 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -0,0 +1,99 @@ +/* + * 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 + +import scala.util.control.NonFatal + +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} +import org.apache.spark.sql.types.StructType + +trait SQLQueryTestHelper { + + private val notIncludedMsg = "[not included in comparison]" + private val clsName = this.getClass.getCanonicalName + protected val emptySchema = StructType(Seq.empty).catalogString + + protected def replaceNotIncludedMsg(line: String): String = { + line.replaceAll("#\\d+", "#x") + .replaceAll( + s"Location.*$clsName/", + s"Location $notIncludedMsg/{warehouse_dir}/") + .replaceAll("Created By.*", s"Created By $notIncludedMsg") + .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") + .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") + .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") + .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds + } + + /** Executes a query and returns the result as (schema of the output, normalized output). */ + protected def getNormalizedResult(session: SparkSession, sql: String): (String, Seq[String]) = { + // Returns true if the plan is supposed to be sorted. + def isSorted(plan: LogicalPlan): Boolean = plan match { + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false + case _: DescribeCommandBase + | _: DescribeColumnCommand + | _: DescribeRelation + | _: DescribeColumn => true + case PhysicalOperation(_, _, Sort(_, true, _)) => true + case _ => plan.children.iterator.exists(isSorted) + } + + val df = session.sql(sql) + val schema = df.schema.catalogString + // Get answer, but also get rid of the #1234 expression ids that show up in explain plans + val answer = SQLExecution.withNewExecutionId(df.queryExecution, Some(sql)) { + hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) + } + + // If the output is not pre-sorted, sort it. + if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) + } + + /** + * This method handles exceptions occurred during query execution as they may need special care + * to become comparable to the expected output. + * + * @param result a function that returns a pair of schema and output + */ + protected def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { + try { + result + } catch { + case a: AnalysisException => + // Do not output the logical plan tree which contains expression IDs. + // Also implement a crude way of masking expression IDs in the error message + // with a generic pattern "###". + val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage + (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) + case s: SparkException if s.getCause != null => + // For a runtime exception, it is hard to match because its message contains + // information of stage, task ID, etc. + // To make result matching simpler, here we match the cause of the exception if it exists. + val cause = s.getCause + (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) + case NonFatal(e) => + // If there is an exception, put the exception class followed by the message. + (emptySchema, Seq(e.getClass.getName, e.getMessage)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala index 02c6fba9725d3..b7b3601de7876 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala @@ -22,22 +22,16 @@ import java.net.URI import java.util.Locale import scala.collection.mutable.ArrayBuffer -import scala.util.control.NonFatal -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator -import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile} import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND -import org.apache.spark.sql.execution.{SQLExecution, WholeStageCodegenExec} -import org.apache.spark.sql.execution.HiveResult.hiveResultString -import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} +import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -import org.apache.spark.sql.types.StructType import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.util.Utils @@ -126,7 +120,8 @@ import org.apache.spark.util.Utils * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ @ExtendedSQLTest -class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper { +class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper + with SQLQueryTestHelper { import IntegratedUDFTestUtils._ @@ -145,11 +140,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper protected val validFileExtensions = ".sql" - private val notIncludedMsg = "[not included in comparison]" - private val clsName = this.getClass.getCanonicalName - - protected val emptySchema = StructType(Seq.empty).catalogString - protected override def sparkConf: SparkConf = super.sparkConf // Fewer shuffle partitions to speed up testing. .set(SQLConf.SHUFFLE_PARTITIONS, 4) @@ -466,70 +456,6 @@ class SQLQueryTestSuite extends QueryTest with SharedSparkSession with SQLHelper } } - /** - * This method handles exceptions occurred during query execution as they may need special care - * to become comparable to the expected output. - * - * @param result a function that returns a pair of schema and output - */ - protected def handleExceptions(result: => (String, Seq[String])): (String, Seq[String]) = { - try { - result - } catch { - case a: AnalysisException => - // Do not output the logical plan tree which contains expression IDs. - // Also implement a crude way of masking expression IDs in the error message - // with a generic pattern "###". - val msg = if (a.plan.nonEmpty) a.getSimpleMessage else a.getMessage - (emptySchema, Seq(a.getClass.getName, msg.replaceAll("#\\d+", "#x"))) - case s: SparkException if s.getCause != null => - // For a runtime exception, it is hard to match because its message contains - // information of stage, task ID, etc. - // To make result matching simpler, here we match the cause of the exception if it exists. - val cause = s.getCause - (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) - case NonFatal(e) => - // If there is an exception, put the exception class followed by the message. - (emptySchema, Seq(e.getClass.getName, e.getMessage)) - } - } - - /** Executes a query and returns the result as (schema of the output, normalized output). */ - private def getNormalizedResult(session: SparkSession, sql: String): (String, Seq[String]) = { - // Returns true if the plan is supposed to be sorted. - def isSorted(plan: LogicalPlan): Boolean = plan match { - case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case _: DescribeCommandBase - | _: DescribeColumnCommand - | _: DescribeRelation - | _: DescribeColumn => true - case PhysicalOperation(_, _, Sort(_, true, _)) => true - case _ => plan.children.iterator.exists(isSorted) - } - - val df = session.sql(sql) - val schema = df.schema.catalogString - // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = SQLExecution.withNewExecutionId(df.queryExecution, Some(sql)) { - hiveResultString(df.queryExecution.executedPlan).map(replaceNotIncludedMsg) - } - - // If the output is not pre-sorted, sort it. - if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted) - } - - protected def replaceNotIncludedMsg(line: String): String = { - line.replaceAll("#\\d+", "#x") - .replaceAll( - s"Location.*$clsName/", - s"Location $notIncludedMsg/{warehouse_dir}/") - .replaceAll("Created By.*", s"Created By $notIncludedMsg") - .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") - .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") - .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") - .replaceAll("\\*\\(\\d+\\) ", "*") // remove the WholeStageCodegen codegenStageIds - } - protected lazy val listTestCases: Seq[TestCase] = { listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 1f16bb69b3a16..c729e1ed9a134 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -17,19 +17,24 @@ package org.apache.spark.sql +import scala.collection.JavaConverters._ + import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} import org.apache.spark.internal.config.EXECUTOR_ALLOW_SPARK_CONTEXT import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.sql.util.ExecutionListenerBus import org.apache.spark.util.ThreadUtils /** * Test cases for the builder pattern of [[SparkSession]]. */ -class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { +class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach with Eventually { override def afterEach(): Unit = { // This suite should not interfere with the other test suites. @@ -39,6 +44,34 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { SparkSession.clearDefaultSession() } + test("SPARK-34087: Fix memory leak of ExecutionListenerBus") { + val spark = SparkSession.builder() + .master("local") + .getOrCreate() + + @inline def listenersNum(): Int = { + spark.sparkContext + .listenerBus + .listeners + .asScala + .count(_.isInstanceOf[ExecutionListenerBus]) + } + + (1 to 10).foreach { _ => + spark.cloneSession() + SparkSession.clearActiveSession() + } + + eventually(timeout(10.seconds), interval(1.seconds)) { + System.gc() + // After GC, the number of ExecutionListenerBus should be less than 11 (we created 11 + // SparkSessions in total). + // Since GC can't 100% guarantee all out-of-referenced objects be cleaned at one time, + // here, we check at least one listener is cleaned up to prove the mechanism works. + assert(listenersNum() < 11) + } + } + test("create with config options and propagate them to SparkContext and SparkSession") { val session = SparkSession.builder() .master("local") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index 73b23496de515..fafe1bb39336f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -542,7 +542,7 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark sql("select a, (select sum(b) from l l2 where l2.a < l1.a) sum_b from l l1") } assert(msg1.getMessage.contains( - "Correlated column is not allowed in a non-equality predicate:")) + "Correlated column is not allowed in predicate (l2.`a` < outer(l1.`a`))")) } test("disjunctive correlated scalar subquery") { @@ -1753,4 +1753,13 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark } } } + + test("SPARK-35080: correlated equality predicates contain only outer references") { + withTempView("t") { + Seq((0, 1), (1, 1)).toDF("c1", "c2").createOrReplaceTempView("t") + checkAnswer( + sql("select c1, c2, (select count(*) from l where c1 = c2) from t"), + Row(0, 1, 0) :: Row(1, 1, 8) :: Nil) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala index a456e08648d0e..b9537d3cd19c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSBase.scala @@ -21,10 +21,10 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession -trait TPCDSBase extends SharedSparkSession { +trait TPCDSBase extends SharedSparkSession with TPCDSSchema { // The TPCDS queries below are based on v1.4 - val tpcdsQueries = Seq( + def tpcdsQueries: Seq[String] = Seq( "q1", "q2", "q3", "q4", "q5", "q6", "q7", "q8", "q9", "q10", "q11", "q12", "q13", "q14a", "q14b", "q15", "q16", "q17", "q18", "q19", "q20", "q21", "q22", "q23a", "q23b", "q24a", "q24b", "q25", "q26", "q27", "q28", "q29", "q30", @@ -48,505 +48,6 @@ trait TPCDSBase extends SharedSparkSession { "q3", "q7", "q10", "q19", "q27", "q34", "q42", "q43", "q46", "q52", "q53", "q55", "q59", "q63", "q65", "q68", "q73", "q79", "q89", "q98", "ss_max") - private val tableColumns = Map( - "store_sales" -> - """ - |`ss_sold_date_sk` INT, - |`ss_sold_time_sk` INT, - |`ss_item_sk` INT, - |`ss_customer_sk` INT, - |`ss_cdemo_sk` INT, - |`ss_hdemo_sk` INT, - |`ss_addr_sk` INT, - |`ss_store_sk` INT, - |`ss_promo_sk` INT, - |`ss_ticket_number` INT, - |`ss_quantity` INT, - |`ss_wholesale_cost` DECIMAL(7,2), - |`ss_list_price` DECIMAL(7,2), - |`ss_sales_price` DECIMAL(7,2), - |`ss_ext_discount_amt` DECIMAL(7,2), - |`ss_ext_sales_price` DECIMAL(7,2), - |`ss_ext_wholesale_cost` DECIMAL(7,2), - |`ss_ext_list_price` DECIMAL(7,2), - |`ss_ext_tax` DECIMAL(7,2), - |`ss_coupon_amt` DECIMAL(7,2), - |`ss_net_paid` DECIMAL(7,2), - |`ss_net_paid_inc_tax` DECIMAL(7,2), - |`ss_net_profit` DECIMAL(7,2) - """.stripMargin, - "store_returns" -> - """ - |`sr_returned_date_sk` BIGINT, - |`sr_return_time_sk` BIGINT, - |`sr_item_sk` BIGINT, - |`sr_customer_sk` BIGINT, - |`sr_cdemo_sk` BIGINT, - |`sr_hdemo_sk` BIGINT, - |`sr_addr_sk` BIGINT, - |`sr_store_sk` BIGINT, - |`sr_reason_sk` BIGINT, - |`sr_ticket_number` BIGINT, - |`sr_return_quantity` INT, - |`sr_return_amt` DECIMAL(7,2), - |`sr_return_tax` DECIMAL(7,2), - |`sr_return_amt_inc_tax` DECIMAL(7,2), - |`sr_fee` DECIMAL(7,2), - |`sr_return_ship_cost` DECIMAL(7,2), - |`sr_refunded_cash` DECIMAL(7,2), - |`sr_reversed_charge` DECIMAL(7,2), - |`sr_store_credit` DECIMAL(7,2), - |`sr_net_loss` DECIMAL(7,2) - """.stripMargin, - "catalog_sales" -> - """ - |`cs_sold_date_sk` INT, - |`cs_sold_time_sk` INT, - |`cs_ship_date_sk` INT, - |`cs_bill_customer_sk` INT, - |`cs_bill_cdemo_sk` INT, - |`cs_bill_hdemo_sk` INT, - |`cs_bill_addr_sk` INT, - |`cs_ship_customer_sk` INT, - |`cs_ship_cdemo_sk` INT, - |`cs_ship_hdemo_sk` INT, - |`cs_ship_addr_sk` INT, - |`cs_call_center_sk` INT, - |`cs_catalog_page_sk` INT, - |`cs_ship_mode_sk` INT, - |`cs_warehouse_sk` INT, - |`cs_item_sk` INT, - |`cs_promo_sk` INT, - |`cs_order_number` INT, - |`cs_quantity` INT, - |`cs_wholesale_cost` DECIMAL(7,2), - |`cs_list_price` DECIMAL(7,2), - |`cs_sales_price` DECIMAL(7,2), - |`cs_ext_discount_amt` DECIMAL(7,2), - |`cs_ext_sales_price` DECIMAL(7,2), - |`cs_ext_wholesale_cost` DECIMAL(7,2), - |`cs_ext_list_price` DECIMAL(7,2), - |`cs_ext_tax` DECIMAL(7,2), - |`cs_coupon_amt` DECIMAL(7,2), - |`cs_ext_ship_cost` DECIMAL(7,2), - |`cs_net_paid` DECIMAL(7,2), - |`cs_net_paid_inc_tax` DECIMAL(7,2), - |`cs_net_paid_inc_ship` DECIMAL(7,2), - |`cs_net_paid_inc_ship_tax` DECIMAL(7,2), - |`cs_net_profit` DECIMAL(7,2) - """.stripMargin, - "catalog_returns" -> - """ - |`cr_returned_date_sk` INT, - |`cr_returned_time_sk` INT, - |`cr_item_sk` INT, - |`cr_refunded_customer_sk` INT, - |`cr_refunded_cdemo_sk` INT, - |`cr_refunded_hdemo_sk` INT, - |`cr_refunded_addr_sk` INT, - |`cr_returning_customer_sk` INT, - |`cr_returning_cdemo_sk` INT, - |`cr_returning_hdemo_sk` INT, - |`cr_returning_addr_sk` INT, - |`cr_call_center_sk` INT, - |`cr_catalog_page_sk` INT, - |`cr_ship_mode_sk` INT, - |`cr_warehouse_sk` INT, - |`cr_reason_sk` INT,`cr_order_number` INT, - |`cr_return_quantity` INT, - |`cr_return_amount` DECIMAL(7,2), - |`cr_return_tax` DECIMAL(7,2), - |`cr_return_amt_inc_tax` DECIMAL(7,2), - |`cr_fee` DECIMAL(7,2), - |`cr_return_ship_cost` DECIMAL(7,2), - |`cr_refunded_cash` DECIMAL(7,2), - |`cr_reversed_charge` DECIMAL(7,2), - |`cr_store_credit` DECIMAL(7,2), - |`cr_net_loss` DECIMAL(7,2) - """.stripMargin, - "web_sales" -> - """ - |`ws_sold_date_sk` INT, - |`ws_sold_time_sk` INT, - |`ws_ship_date_sk` INT, - |`ws_item_sk` INT, - |`ws_bill_customer_sk` INT, - |`ws_bill_cdemo_sk` INT, - |`ws_bill_hdemo_sk` INT, - |`ws_bill_addr_sk` INT, - |`ws_ship_customer_sk` INT, - |`ws_ship_cdemo_sk` INT, - |`ws_ship_hdemo_sk` INT, - |`ws_ship_addr_sk` INT, - |`ws_web_page_sk` INT, - |`ws_web_site_sk` INT, - |`ws_ship_mode_sk` INT, - |`ws_warehouse_sk` INT, - |`ws_promo_sk` INT, - |`ws_order_number` INT, - |`ws_quantity` INT, - |`ws_wholesale_cost` DECIMAL(7,2), - |`ws_list_price` DECIMAL(7,2), - |`ws_sales_price` DECIMAL(7,2), - |`ws_ext_discount_amt` DECIMAL(7,2), - |`ws_ext_sales_price` DECIMAL(7,2), - |`ws_ext_wholesale_cost` DECIMAL(7,2), - |`ws_ext_list_price` DECIMAL(7,2), - |`ws_ext_tax` DECIMAL(7,2), - |`ws_coupon_amt` DECIMAL(7,2), - |`ws_ext_ship_cost` DECIMAL(7,2), - |`ws_net_paid` DECIMAL(7,2), - |`ws_net_paid_inc_tax` DECIMAL(7,2), - |`ws_net_paid_inc_ship` DECIMAL(7,2), - |`ws_net_paid_inc_ship_tax` DECIMAL(7,2), - |`ws_net_profit` DECIMAL(7,2) - """.stripMargin, - "web_returns" -> - """ - |`wr_returned_date_sk` BIGINT, - |`wr_returned_time_sk` BIGINT, - |`wr_item_sk` BIGINT, - |`wr_refunded_customer_sk` BIGINT, - |`wr_refunded_cdemo_sk` BIGINT, - |`wr_refunded_hdemo_sk` BIGINT, - |`wr_refunded_addr_sk` BIGINT, - |`wr_returning_customer_sk` BIGINT, - |`wr_returning_cdemo_sk` BIGINT, - |`wr_returning_hdemo_sk` BIGINT, - |`wr_returning_addr_sk` BIGINT, - |`wr_web_page_sk` BIGINT, - |`wr_reason_sk` BIGINT, - |`wr_order_number` BIGINT, - |`wr_return_quantity` INT, - |`wr_return_amt` DECIMAL(7,2), - |`wr_return_tax` DECIMAL(7,2), - |`wr_return_amt_inc_tax` DECIMAL(7,2), - |`wr_fee` DECIMAL(7,2), - |`wr_return_ship_cost` DECIMAL(7,2), - |`wr_refunded_cash` DECIMAL(7,2), - |`wr_reversed_charge` DECIMAL(7,2), - |`wr_account_credit` DECIMAL(7,2), - |`wr_net_loss` DECIMAL(7,2) - """.stripMargin, - "inventory" -> - """ - |`inv_date_sk` INT, - |`inv_item_sk` INT, - |`inv_warehouse_sk` INT, - |`inv_quantity_on_hand` INT - """.stripMargin, - "store" -> - """ - |`s_store_sk` INT, - |`s_store_id` CHAR(16), - |`s_rec_start_date` DATE, - |`s_rec_end_date` DATE, - |`s_closed_date_sk` INT, - |`s_store_name` VARCHAR(50), - |`s_number_employees` INT, - |`s_floor_space` INT, - |`s_hours` CHAR(20), - |`s_manager` VARCHAR(40), - |`s_market_id` INT, - |`s_geography_class` VARCHAR(100), - |`s_market_desc` VARCHAR(100), - |`s_market_manager` VARCHAR(40), - |`s_division_id` INT, - |`s_division_name` VARCHAR(50), - |`s_company_id` INT, - |`s_company_name` VARCHAR(50), - |`s_street_number` VARCHAR(10), - |`s_street_name` VARCHAR(60), - |`s_street_type` CHAR(15), - |`s_suite_number` CHAR(10), - |`s_city` VARCHAR(60), - |`s_county` VARCHAR(30), - |`s_state` CHAR(2), - |`s_zip` CHAR(10), - |`s_country` VARCHAR(20), - |`s_gmt_offset` DECIMAL(5,2), - |`s_tax_percentage` DECIMAL(5,2) - """.stripMargin, - "call_center" -> - """ - |`cc_call_center_sk` INT, - |`cc_call_center_id` CHAR(16), - |`cc_rec_start_date` DATE, - |`cc_rec_end_date` DATE, - |`cc_closed_date_sk` INT, - |`cc_open_date_sk` INT, - |`cc_name` VARCHAR(50), - |`cc_class` VARCHAR(50), - |`cc_employees` INT, - |`cc_sq_ft` INT, - |`cc_hours` CHAR(20), - |`cc_manager` VARCHAR(40), - |`cc_mkt_id` INT, - |`cc_mkt_class` CHAR(50), - |`cc_mkt_desc` VARCHAR(100), - |`cc_market_manager` VARCHAR(40), - |`cc_division` INT, - |`cc_division_name` VARCHAR(50), - |`cc_company` INT, - |`cc_company_name` CHAR(50), - |`cc_street_number` CHAR(10), - |`cc_street_name` VARCHAR(60), - |`cc_street_type` CHAR(15), - |`cc_suite_number` CHAR(10), - |`cc_city` VARCHAR(60), - |`cc_county` VARCHAR(30), - |`cc_state` CHAR(2), - |`cc_zip` CHAR(10), - |`cc_country` VARCHAR(20), - |`cc_gmt_offset` DECIMAL(5,2), - |`cc_tax_percentage` DECIMAL(5,2) - """.stripMargin, - "catalog_page" -> - """ - |`cp_catalog_page_sk` INT, - |`cp_catalog_page_id` CHAR(16), - |`cp_start_date_sk` INT, - |`cp_end_date_sk` INT, - |`cp_department` VARCHAR(50), - |`cp_catalog_number` INT, - |`cp_catalog_page_number` INT, - |`cp_description` VARCHAR(100), - |`cp_type` VARCHAR(100) - """.stripMargin, - "web_site" -> - """ - |`web_site_sk` INT, - |`web_site_id` CHAR(16), - |`web_rec_start_date` DATE, - |`web_rec_end_date` DATE, - |`web_name` VARCHAR(50), - |`web_open_date_sk` INT, - |`web_close_date_sk` INT, - |`web_class` VARCHAR(50), - |`web_manager` VARCHAR(40), - |`web_mkt_id` INT, - |`web_mkt_class` VARCHAR(50), - |`web_mkt_desc` VARCHAR(100), - |`web_market_manager` VARCHAR(40), - |`web_company_id` INT, - |`web_company_name` CHAR(50), - |`web_street_number` CHAR(10), - |`web_street_name` VARCHAR(60), - |`web_street_type` CHAR(15), - |`web_suite_number` CHAR(10), - |`web_city` VARCHAR(60), - |`web_county` VARCHAR(30), - |`web_state` CHAR(2), - |`web_zip` CHAR(10), - |`web_country` VARCHAR(20), - |`web_gmt_offset` DECIMAL(5,2), - |`web_tax_percentage` DECIMAL(5,2) - """.stripMargin, - "web_page" -> - """ - |`wp_web_page_sk` INT, - |`wp_web_page_id` CHAR(16), - |`wp_rec_start_date` DATE, - |`wp_rec_end_date` DATE, - |`wp_creation_date_sk` INT, - |`wp_access_date_sk` INT, - |`wp_autogen_flag` CHAR(1), - |`wp_customer_sk` INT, - |`wp_url` VARCHAR(100), - |`wp_type` CHAR(50), - |`wp_char_count` INT, - |`wp_link_count` INT, - |`wp_image_count` INT, - |`wp_max_ad_count` INT - """.stripMargin, - "warehouse" -> - """ - |`w_warehouse_sk` INT, - |`w_warehouse_id` CHAR(16), - |`w_warehouse_name` VARCHAR(20), - |`w_warehouse_sq_ft` INT, - |`w_street_number` CHAR(10), - |`w_street_name` VARCHAR(20), - |`w_street_type` CHAR(15), - |`w_suite_number` CHAR(10), - |`w_city` VARCHAR(60), - |`w_county` VARCHAR(30), - |`w_state` CHAR(2), - |`w_zip` CHAR(10), - |`w_country` VARCHAR(20), - |`w_gmt_offset` DECIMAL(5,2) - """.stripMargin, - "customer" -> - """ - |`c_customer_sk` INT, - |`c_customer_id` CHAR(16), - |`c_current_cdemo_sk` INT, - |`c_current_hdemo_sk` INT, - |`c_current_addr_sk` INT, - |`c_first_shipto_date_sk` INT, - |`c_first_sales_date_sk` INT, - |`c_salutation` CHAR(10), - |`c_first_name` CHAR(20), - |`c_last_name` CHAR(30), - |`c_preferred_cust_flag` CHAR(1), - |`c_birth_day` INT, - |`c_birth_month` INT, - |`c_birth_year` INT, - |`c_birth_country` VARCHAR(20), - |`c_login` CHAR(13), - |`c_email_address` CHAR(50), - |`c_last_review_date` INT - """.stripMargin, - "customer_address" -> - """ - |`ca_address_sk` INT, - |`ca_address_id` CHAR(16), - |`ca_street_number` CHAR(10), - |`ca_street_name` VARCHAR(60), - |`ca_street_type` CHAR(15), - |`ca_suite_number` CHAR(10), - |`ca_city` VARCHAR(60), - |`ca_county` VARCHAR(30), - |`ca_state` CHAR(2), - |`ca_zip` CHAR(10), - |`ca_country` VARCHAR(20), - |`ca_gmt_offset` DECIMAL(5,2), - |`ca_location_type` CHAR(20) - """.stripMargin, - "customer_demographics" -> - """ - |`cd_demo_sk` INT, - |`cd_gender` CHAR(1), - |`cd_marital_status` CHAR(1), - |`cd_education_status` CHAR(20), - |`cd_purchase_estimate` INT, - |`cd_credit_rating` CHAR(10), - |`cd_dep_count` INT, - |`cd_dep_employed_count` INT, - |`cd_dep_college_count` INT - """.stripMargin, - "date_dim" -> - """ - |`d_date_sk` INT, - |`d_date_id` CHAR(16), - |`d_date` DATE, - |`d_month_seq` INT, - |`d_week_seq` INT, - |`d_quarter_seq` INT, - |`d_year` INT, - |`d_dow` INT, - |`d_moy` INT, - |`d_dom` INT, - |`d_qoy` INT, - |`d_fy_year` INT, - |`d_fy_quarter_seq` INT, - |`d_fy_week_seq` INT, - |`d_day_name` CHAR(9), - |`d_quarter_name` CHAR(1), - |`d_holiday` CHAR(1), - |`d_weekend` CHAR(1), - |`d_following_holiday` CHAR(1), - |`d_first_dom` INT, - |`d_last_dom` INT, - |`d_same_day_ly` INT, - |`d_same_day_lq` INT, - |`d_current_day` CHAR(1), - |`d_current_week` CHAR(1), - |`d_current_month` CHAR(1), - |`d_current_quarter` CHAR(1), - |`d_current_year` CHAR(1) - """.stripMargin, - "household_demographics" -> - """ - |`hd_demo_sk` INT, - |`hd_income_band_sk` INT, - |`hd_buy_potential` CHAR(15), - |`hd_dep_count` INT, - |`hd_vehicle_count` INT - """.stripMargin, - "item" -> - """ - |`i_item_sk` INT, - |`i_item_id` CHAR(16), - |`i_rec_start_date` DATE, - |`i_rec_end_date` DATE, - |`i_item_desc` VARCHAR(200), - |`i_current_price` DECIMAL(7,2), - |`i_wholesale_cost` DECIMAL(7,2), - |`i_brand_id` INT, - |`i_brand` CHAR(50), - |`i_class_id` INT, - |`i_class` CHAR(50), - |`i_category_id` INT, - |`i_category` CHAR(50), - |`i_manufact_id` INT, - |`i_manufact` CHAR(50), - |`i_size` CHAR(20), - |`i_formulation` CHAR(20), - |`i_color` CHAR(20), - |`i_units` CHAR(10), - |`i_container` CHAR(10), - |`i_manager_id` INT, - |`i_product_name` CHAR(50) - """.stripMargin, - "income_band" -> - """ - |`ib_income_band_sk` INT, - |`ib_lower_bound` INT, - |`ib_upper_bound` INT - """.stripMargin, - "promotion" -> - """ - |`p_promo_sk` INT, - |`p_promo_id` CHAR(16), - |`p_start_date_sk` INT, - |`p_end_date_sk` INT, - |`p_item_sk` INT, - |`p_cost` DECIMAL(15,2), - |`p_response_target` INT, - |`p_promo_name` CHAR(50), - |`p_channel_dmail` CHAR(1), - |`p_channel_email` CHAR(1), - |`p_channel_catalog` CHAR(1), - |`p_channel_tv` CHAR(1), - |`p_channel_radio` CHAR(1), - |`p_channel_press` CHAR(1), - |`p_channel_event` CHAR(1), - |`p_channel_demo` CHAR(1), - |`p_channel_details` VARCHAR(100), - |`p_purpose` CHAR(15), - |`p_discount_active` CHAR(1) - """.stripMargin, - "reason" -> - """ - |`r_reason_sk` INT, - |`r_reason_id` CHAR(16), - |`r_reason_desc` CHAR(100) - """.stripMargin, - "ship_mode" -> - """ - |`sm_ship_mode_sk` INT, - |`sm_ship_mode_id` CHAR(16), - |`sm_type` CHAR(30), - |`sm_code` CHAR(10), - |`sm_carrier` CHAR(20), - |`sm_contract` CHAR(20) - """.stripMargin, - "time_dim" -> - """ - |`t_time_sk` INT, - |`t_time_id` CHAR(16), - |`t_time` INT, - |`t_hour` INT, - |`t_minute` INT, - |`t_second` INT, - |`t_am_pm` CHAR(2), - |`t_shift` CHAR(20), - |`t_sub_shift` CHAR(20), - |`t_meal_time` CHAR(20) - """.stripMargin - ) - val tableNames: Iterable[String] = tableColumns.keys def createTable( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala new file mode 100644 index 0000000000000..670048826e3bf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala @@ -0,0 +1,164 @@ +/* + * 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 + +import java.io.File +import java.nio.file.{Files, Paths} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.catalyst.util.{fileToString, resourceToString, stringToFile} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.TestSparkSession + +/** + * End-to-end tests to check TPCDS query results. + * + * To run this test suite: + * {{{ + * SPARK_TPCDS_DATA= build/sbt "sql/testOnly *TPCDSQueryTestSuite" + * }}} + * + * To run a single test file upon change: + * {{{ + * SPARK_TPCDS_DATA= + * build/sbt "~sql/testOnly *TPCDSQueryTestSuite -- -z q79" + * }}} + * + * To re-generate golden files for this suite, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 SPARK_TPCDS_DATA= + * build/sbt "sql/testOnly *TPCDSQueryTestSuite" + * }}} + * + * To re-generate golden file for a single test, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 SPARK_TPCDS_DATA= + * build/sbt "sql/testOnly *TPCDSQueryTestSuite -- -z q79" + * }}} + */ +class TPCDSQueryTestSuite extends QueryTest with TPCDSBase with SQLQueryTestHelper { + + private val tpcdsDataPath = sys.env.get("SPARK_TPCDS_DATA") + private val regenerateGoldenFiles = sys.env.get("SPARK_GENERATE_GOLDEN_FILES").exists(_ == "1") + + // To make output results deterministic + protected override def sparkConf: SparkConf = super.sparkConf + .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + + protected override def createSparkSession: TestSparkSession = { + new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) + } + + // We use SF=1 table data here, so we cannot use SF=100 stats + protected override val injectStats: Boolean = false + + if (tpcdsDataPath.nonEmpty) { + val nonExistentTables = tableNames.filterNot { tableName => + Files.exists(Paths.get(s"${tpcdsDataPath.get}/$tableName")) + } + if (nonExistentTables.nonEmpty) { + fail(s"Non-existent TPCDS table paths found in ${tpcdsDataPath.get}: " + + nonExistentTables.mkString(", ")) + } + } + + protected val baseResourcePath = { + // use the same way as `SQLQueryTestSuite` to get the resource path + java.nio.file.Paths.get("src", "test", "resources", "tpcds-query-results") + .toFile.getAbsolutePath + } + + override val tpcdsQueries = { + // SPARK-35327: Filters out the TPC-DS queries that can cause flaky test results + val excludedQueries = Set("q6", "q75") + super.tpcdsQueries.filterNot(excludedQueries.contains) + } + + override def createTable( + spark: SparkSession, + tableName: String, + format: String = "parquet", + options: Seq[String] = Nil): Unit = { + spark.sql( + s""" + |CREATE TABLE `$tableName` (${tableColumns(tableName)}) + |USING $format + |LOCATION '${tpcdsDataPath.get}/$tableName' + |${options.mkString("\n")} + """.stripMargin) + } + + private def runQuery(query: String, goldenFile: File): Unit = { + val (schema, output) = handleExceptions(getNormalizedResult(spark, query)) + val queryString = query.trim + val outputString = output.mkString("\n").replaceAll("\\s+$", "") + if (regenerateGoldenFiles) { + val goldenOutput = { + s"-- Automatically generated by ${getClass.getSimpleName}\n\n" + + s"-- !query schema\n" + + schema + "\n" + + s"-- !query output\n" + + outputString + + "\n" + } + val parent = goldenFile.getParentFile + if (!parent.exists()) { + assert(parent.mkdirs(), "Could not create directory: " + parent) + } + stringToFile(goldenFile, goldenOutput) + } + + // Read back the golden file. + val (expectedSchema, expectedOutput) = { + val goldenOutput = fileToString(goldenFile) + val segments = goldenOutput.split("-- !query.*\n") + + // query has 3 segments, plus the header + assert(segments.size == 3, + s"Expected 3 blocks in result file but got ${segments.size}. " + + "Try regenerate the result files.") + + (segments(1).trim, segments(2).replaceAll("\\s+$", "")) + } + + assertResult(expectedSchema, s"Schema did not match\n$queryString") { schema } + assertResult(expectedOutput, s"Result did not match\n$queryString") { outputString } + } + + if (tpcdsDataPath.nonEmpty) { + tpcdsQueries.foreach { name => + val queryString = resourceToString(s"tpcds/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(name) { + val goldenFile = new File(s"$baseResourcePath/v1_4", s"$name.sql.out") + runQuery(queryString, goldenFile) + } + } + + tpcdsQueriesV2_7_0.foreach { name => + val queryString = resourceToString(s"tpcds-v2.7.0/$name.sql", + classLoader = Thread.currentThread().getContextClassLoader) + test(s"$name-v2.7") { + val goldenFile = new File(s"$baseResourcePath/v2_7", s"$name.sql.out") + runQuery(queryString, goldenFile) + } + } + } else { + ignore("skipped because env `SPARK_TPCDS_DATA` is not set") {} + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala new file mode 100644 index 0000000000000..b3543aa4471c5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSSchema.scala @@ -0,0 +1,520 @@ +/* + * 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 + +trait TPCDSSchema { + + protected val tableColumns: Map[String, String] = Map( + "store_sales" -> + """ + |`ss_sold_date_sk` INT, + |`ss_sold_time_sk` INT, + |`ss_item_sk` INT, + |`ss_customer_sk` INT, + |`ss_cdemo_sk` INT, + |`ss_hdemo_sk` INT, + |`ss_addr_sk` INT, + |`ss_store_sk` INT, + |`ss_promo_sk` INT, + |`ss_ticket_number` INT, + |`ss_quantity` INT, + |`ss_wholesale_cost` DECIMAL(7,2), + |`ss_list_price` DECIMAL(7,2), + |`ss_sales_price` DECIMAL(7,2), + |`ss_ext_discount_amt` DECIMAL(7,2), + |`ss_ext_sales_price` DECIMAL(7,2), + |`ss_ext_wholesale_cost` DECIMAL(7,2), + |`ss_ext_list_price` DECIMAL(7,2), + |`ss_ext_tax` DECIMAL(7,2), + |`ss_coupon_amt` DECIMAL(7,2), + |`ss_net_paid` DECIMAL(7,2), + |`ss_net_paid_inc_tax` DECIMAL(7,2), + |`ss_net_profit` DECIMAL(7,2) + """.stripMargin, + "store_returns" -> + """ + |`sr_returned_date_sk` BIGINT, + |`sr_return_time_sk` BIGINT, + |`sr_item_sk` BIGINT, + |`sr_customer_sk` BIGINT, + |`sr_cdemo_sk` BIGINT, + |`sr_hdemo_sk` BIGINT, + |`sr_addr_sk` BIGINT, + |`sr_store_sk` BIGINT, + |`sr_reason_sk` BIGINT, + |`sr_ticket_number` BIGINT, + |`sr_return_quantity` INT, + |`sr_return_amt` DECIMAL(7,2), + |`sr_return_tax` DECIMAL(7,2), + |`sr_return_amt_inc_tax` DECIMAL(7,2), + |`sr_fee` DECIMAL(7,2), + |`sr_return_ship_cost` DECIMAL(7,2), + |`sr_refunded_cash` DECIMAL(7,2), + |`sr_reversed_charge` DECIMAL(7,2), + |`sr_store_credit` DECIMAL(7,2), + |`sr_net_loss` DECIMAL(7,2) + """.stripMargin, + "catalog_sales" -> + """ + |`cs_sold_date_sk` INT, + |`cs_sold_time_sk` INT, + |`cs_ship_date_sk` INT, + |`cs_bill_customer_sk` INT, + |`cs_bill_cdemo_sk` INT, + |`cs_bill_hdemo_sk` INT, + |`cs_bill_addr_sk` INT, + |`cs_ship_customer_sk` INT, + |`cs_ship_cdemo_sk` INT, + |`cs_ship_hdemo_sk` INT, + |`cs_ship_addr_sk` INT, + |`cs_call_center_sk` INT, + |`cs_catalog_page_sk` INT, + |`cs_ship_mode_sk` INT, + |`cs_warehouse_sk` INT, + |`cs_item_sk` INT, + |`cs_promo_sk` INT, + |`cs_order_number` INT, + |`cs_quantity` INT, + |`cs_wholesale_cost` DECIMAL(7,2), + |`cs_list_price` DECIMAL(7,2), + |`cs_sales_price` DECIMAL(7,2), + |`cs_ext_discount_amt` DECIMAL(7,2), + |`cs_ext_sales_price` DECIMAL(7,2), + |`cs_ext_wholesale_cost` DECIMAL(7,2), + |`cs_ext_list_price` DECIMAL(7,2), + |`cs_ext_tax` DECIMAL(7,2), + |`cs_coupon_amt` DECIMAL(7,2), + |`cs_ext_ship_cost` DECIMAL(7,2), + |`cs_net_paid` DECIMAL(7,2), + |`cs_net_paid_inc_tax` DECIMAL(7,2), + |`cs_net_paid_inc_ship` DECIMAL(7,2), + |`cs_net_paid_inc_ship_tax` DECIMAL(7,2), + |`cs_net_profit` DECIMAL(7,2) + """.stripMargin, + "catalog_returns" -> + """ + |`cr_returned_date_sk` INT, + |`cr_returned_time_sk` INT, + |`cr_item_sk` INT, + |`cr_refunded_customer_sk` INT, + |`cr_refunded_cdemo_sk` INT, + |`cr_refunded_hdemo_sk` INT, + |`cr_refunded_addr_sk` INT, + |`cr_returning_customer_sk` INT, + |`cr_returning_cdemo_sk` INT, + |`cr_returning_hdemo_sk` INT, + |`cr_returning_addr_sk` INT, + |`cr_call_center_sk` INT, + |`cr_catalog_page_sk` INT, + |`cr_ship_mode_sk` INT, + |`cr_warehouse_sk` INT, + |`cr_reason_sk` INT,`cr_order_number` INT, + |`cr_return_quantity` INT, + |`cr_return_amount` DECIMAL(7,2), + |`cr_return_tax` DECIMAL(7,2), + |`cr_return_amt_inc_tax` DECIMAL(7,2), + |`cr_fee` DECIMAL(7,2), + |`cr_return_ship_cost` DECIMAL(7,2), + |`cr_refunded_cash` DECIMAL(7,2), + |`cr_reversed_charge` DECIMAL(7,2), + |`cr_store_credit` DECIMAL(7,2), + |`cr_net_loss` DECIMAL(7,2) + """.stripMargin, + "web_sales" -> + """ + |`ws_sold_date_sk` INT, + |`ws_sold_time_sk` INT, + |`ws_ship_date_sk` INT, + |`ws_item_sk` INT, + |`ws_bill_customer_sk` INT, + |`ws_bill_cdemo_sk` INT, + |`ws_bill_hdemo_sk` INT, + |`ws_bill_addr_sk` INT, + |`ws_ship_customer_sk` INT, + |`ws_ship_cdemo_sk` INT, + |`ws_ship_hdemo_sk` INT, + |`ws_ship_addr_sk` INT, + |`ws_web_page_sk` INT, + |`ws_web_site_sk` INT, + |`ws_ship_mode_sk` INT, + |`ws_warehouse_sk` INT, + |`ws_promo_sk` INT, + |`ws_order_number` INT, + |`ws_quantity` INT, + |`ws_wholesale_cost` DECIMAL(7,2), + |`ws_list_price` DECIMAL(7,2), + |`ws_sales_price` DECIMAL(7,2), + |`ws_ext_discount_amt` DECIMAL(7,2), + |`ws_ext_sales_price` DECIMAL(7,2), + |`ws_ext_wholesale_cost` DECIMAL(7,2), + |`ws_ext_list_price` DECIMAL(7,2), + |`ws_ext_tax` DECIMAL(7,2), + |`ws_coupon_amt` DECIMAL(7,2), + |`ws_ext_ship_cost` DECIMAL(7,2), + |`ws_net_paid` DECIMAL(7,2), + |`ws_net_paid_inc_tax` DECIMAL(7,2), + |`ws_net_paid_inc_ship` DECIMAL(7,2), + |`ws_net_paid_inc_ship_tax` DECIMAL(7,2), + |`ws_net_profit` DECIMAL(7,2) + """.stripMargin, + "web_returns" -> + """ + |`wr_returned_date_sk` BIGINT, + |`wr_returned_time_sk` BIGINT, + |`wr_item_sk` BIGINT, + |`wr_refunded_customer_sk` BIGINT, + |`wr_refunded_cdemo_sk` BIGINT, + |`wr_refunded_hdemo_sk` BIGINT, + |`wr_refunded_addr_sk` BIGINT, + |`wr_returning_customer_sk` BIGINT, + |`wr_returning_cdemo_sk` BIGINT, + |`wr_returning_hdemo_sk` BIGINT, + |`wr_returning_addr_sk` BIGINT, + |`wr_web_page_sk` BIGINT, + |`wr_reason_sk` BIGINT, + |`wr_order_number` BIGINT, + |`wr_return_quantity` INT, + |`wr_return_amt` DECIMAL(7,2), + |`wr_return_tax` DECIMAL(7,2), + |`wr_return_amt_inc_tax` DECIMAL(7,2), + |`wr_fee` DECIMAL(7,2), + |`wr_return_ship_cost` DECIMAL(7,2), + |`wr_refunded_cash` DECIMAL(7,2), + |`wr_reversed_charge` DECIMAL(7,2), + |`wr_account_credit` DECIMAL(7,2), + |`wr_net_loss` DECIMAL(7,2) + """.stripMargin, + "inventory" -> + """ + |`inv_date_sk` INT, + |`inv_item_sk` INT, + |`inv_warehouse_sk` INT, + |`inv_quantity_on_hand` INT + """.stripMargin, + "store" -> + """ + |`s_store_sk` INT, + |`s_store_id` CHAR(16), + |`s_rec_start_date` DATE, + |`s_rec_end_date` DATE, + |`s_closed_date_sk` INT, + |`s_store_name` VARCHAR(50), + |`s_number_employees` INT, + |`s_floor_space` INT, + |`s_hours` CHAR(20), + |`s_manager` VARCHAR(40), + |`s_market_id` INT, + |`s_geography_class` VARCHAR(100), + |`s_market_desc` VARCHAR(100), + |`s_market_manager` VARCHAR(40), + |`s_division_id` INT, + |`s_division_name` VARCHAR(50), + |`s_company_id` INT, + |`s_company_name` VARCHAR(50), + |`s_street_number` VARCHAR(10), + |`s_street_name` VARCHAR(60), + |`s_street_type` CHAR(15), + |`s_suite_number` CHAR(10), + |`s_city` VARCHAR(60), + |`s_county` VARCHAR(30), + |`s_state` CHAR(2), + |`s_zip` CHAR(10), + |`s_country` VARCHAR(20), + |`s_gmt_offset` DECIMAL(5,2), + |`s_tax_percentage` DECIMAL(5,2) + """.stripMargin, + "call_center" -> + """ + |`cc_call_center_sk` INT, + |`cc_call_center_id` CHAR(16), + |`cc_rec_start_date` DATE, + |`cc_rec_end_date` DATE, + |`cc_closed_date_sk` INT, + |`cc_open_date_sk` INT, + |`cc_name` VARCHAR(50), + |`cc_class` VARCHAR(50), + |`cc_employees` INT, + |`cc_sq_ft` INT, + |`cc_hours` CHAR(20), + |`cc_manager` VARCHAR(40), + |`cc_mkt_id` INT, + |`cc_mkt_class` CHAR(50), + |`cc_mkt_desc` VARCHAR(100), + |`cc_market_manager` VARCHAR(40), + |`cc_division` INT, + |`cc_division_name` VARCHAR(50), + |`cc_company` INT, + |`cc_company_name` CHAR(50), + |`cc_street_number` CHAR(10), + |`cc_street_name` VARCHAR(60), + |`cc_street_type` CHAR(15), + |`cc_suite_number` CHAR(10), + |`cc_city` VARCHAR(60), + |`cc_county` VARCHAR(30), + |`cc_state` CHAR(2), + |`cc_zip` CHAR(10), + |`cc_country` VARCHAR(20), + |`cc_gmt_offset` DECIMAL(5,2), + |`cc_tax_percentage` DECIMAL(5,2) + """.stripMargin, + "catalog_page" -> + """ + |`cp_catalog_page_sk` INT, + |`cp_catalog_page_id` CHAR(16), + |`cp_start_date_sk` INT, + |`cp_end_date_sk` INT, + |`cp_department` VARCHAR(50), + |`cp_catalog_number` INT, + |`cp_catalog_page_number` INT, + |`cp_description` VARCHAR(100), + |`cp_type` VARCHAR(100) + """.stripMargin, + "web_site" -> + """ + |`web_site_sk` INT, + |`web_site_id` CHAR(16), + |`web_rec_start_date` DATE, + |`web_rec_end_date` DATE, + |`web_name` VARCHAR(50), + |`web_open_date_sk` INT, + |`web_close_date_sk` INT, + |`web_class` VARCHAR(50), + |`web_manager` VARCHAR(40), + |`web_mkt_id` INT, + |`web_mkt_class` VARCHAR(50), + |`web_mkt_desc` VARCHAR(100), + |`web_market_manager` VARCHAR(40), + |`web_company_id` INT, + |`web_company_name` CHAR(50), + |`web_street_number` CHAR(10), + |`web_street_name` VARCHAR(60), + |`web_street_type` CHAR(15), + |`web_suite_number` CHAR(10), + |`web_city` VARCHAR(60), + |`web_county` VARCHAR(30), + |`web_state` CHAR(2), + |`web_zip` CHAR(10), + |`web_country` VARCHAR(20), + |`web_gmt_offset` DECIMAL(5,2), + |`web_tax_percentage` DECIMAL(5,2) + """.stripMargin, + "web_page" -> + """ + |`wp_web_page_sk` INT, + |`wp_web_page_id` CHAR(16), + |`wp_rec_start_date` DATE, + |`wp_rec_end_date` DATE, + |`wp_creation_date_sk` INT, + |`wp_access_date_sk` INT, + |`wp_autogen_flag` CHAR(1), + |`wp_customer_sk` INT, + |`wp_url` VARCHAR(100), + |`wp_type` CHAR(50), + |`wp_char_count` INT, + |`wp_link_count` INT, + |`wp_image_count` INT, + |`wp_max_ad_count` INT + """.stripMargin, + "warehouse" -> + """ + |`w_warehouse_sk` INT, + |`w_warehouse_id` CHAR(16), + |`w_warehouse_name` VARCHAR(20), + |`w_warehouse_sq_ft` INT, + |`w_street_number` CHAR(10), + |`w_street_name` VARCHAR(20), + |`w_street_type` CHAR(15), + |`w_suite_number` CHAR(10), + |`w_city` VARCHAR(60), + |`w_county` VARCHAR(30), + |`w_state` CHAR(2), + |`w_zip` CHAR(10), + |`w_country` VARCHAR(20), + |`w_gmt_offset` DECIMAL(5,2) + """.stripMargin, + "customer" -> + """ + |`c_customer_sk` INT, + |`c_customer_id` CHAR(16), + |`c_current_cdemo_sk` INT, + |`c_current_hdemo_sk` INT, + |`c_current_addr_sk` INT, + |`c_first_shipto_date_sk` INT, + |`c_first_sales_date_sk` INT, + |`c_salutation` CHAR(10), + |`c_first_name` CHAR(20), + |`c_last_name` CHAR(30), + |`c_preferred_cust_flag` CHAR(1), + |`c_birth_day` INT, + |`c_birth_month` INT, + |`c_birth_year` INT, + |`c_birth_country` VARCHAR(20), + |`c_login` CHAR(13), + |`c_email_address` CHAR(50), + |`c_last_review_date` INT + """.stripMargin, + "customer_address" -> + """ + |`ca_address_sk` INT, + |`ca_address_id` CHAR(16), + |`ca_street_number` CHAR(10), + |`ca_street_name` VARCHAR(60), + |`ca_street_type` CHAR(15), + |`ca_suite_number` CHAR(10), + |`ca_city` VARCHAR(60), + |`ca_county` VARCHAR(30), + |`ca_state` CHAR(2), + |`ca_zip` CHAR(10), + |`ca_country` VARCHAR(20), + |`ca_gmt_offset` DECIMAL(5,2), + |`ca_location_type` CHAR(20) + """.stripMargin, + "customer_demographics" -> + """ + |`cd_demo_sk` INT, + |`cd_gender` CHAR(1), + |`cd_marital_status` CHAR(1), + |`cd_education_status` CHAR(20), + |`cd_purchase_estimate` INT, + |`cd_credit_rating` CHAR(10), + |`cd_dep_count` INT, + |`cd_dep_employed_count` INT, + |`cd_dep_college_count` INT + """.stripMargin, + "date_dim" -> + """ + |`d_date_sk` INT, + |`d_date_id` CHAR(16), + |`d_date` DATE, + |`d_month_seq` INT, + |`d_week_seq` INT, + |`d_quarter_seq` INT, + |`d_year` INT, + |`d_dow` INT, + |`d_moy` INT, + |`d_dom` INT, + |`d_qoy` INT, + |`d_fy_year` INT, + |`d_fy_quarter_seq` INT, + |`d_fy_week_seq` INT, + |`d_day_name` CHAR(9), + |`d_quarter_name` CHAR(6), + |`d_holiday` CHAR(1), + |`d_weekend` CHAR(1), + |`d_following_holiday` CHAR(1), + |`d_first_dom` INT, + |`d_last_dom` INT, + |`d_same_day_ly` INT, + |`d_same_day_lq` INT, + |`d_current_day` CHAR(1), + |`d_current_week` CHAR(1), + |`d_current_month` CHAR(1), + |`d_current_quarter` CHAR(1), + |`d_current_year` CHAR(1) + """.stripMargin, + "household_demographics" -> + """ + |`hd_demo_sk` INT, + |`hd_income_band_sk` INT, + |`hd_buy_potential` CHAR(15), + |`hd_dep_count` INT, + |`hd_vehicle_count` INT + """.stripMargin, + "item" -> + """ + |`i_item_sk` INT, + |`i_item_id` CHAR(16), + |`i_rec_start_date` DATE, + |`i_rec_end_date` DATE, + |`i_item_desc` VARCHAR(200), + |`i_current_price` DECIMAL(7,2), + |`i_wholesale_cost` DECIMAL(7,2), + |`i_brand_id` INT, + |`i_brand` CHAR(50), + |`i_class_id` INT, + |`i_class` CHAR(50), + |`i_category_id` INT, + |`i_category` CHAR(50), + |`i_manufact_id` INT, + |`i_manufact` CHAR(50), + |`i_size` CHAR(20), + |`i_formulation` CHAR(20), + |`i_color` CHAR(20), + |`i_units` CHAR(10), + |`i_container` CHAR(10), + |`i_manager_id` INT, + |`i_product_name` CHAR(50) + """.stripMargin, + "income_band" -> + """ + |`ib_income_band_sk` INT, + |`ib_lower_bound` INT, + |`ib_upper_bound` INT + """.stripMargin, + "promotion" -> + """ + |`p_promo_sk` INT, + |`p_promo_id` CHAR(16), + |`p_start_date_sk` INT, + |`p_end_date_sk` INT, + |`p_item_sk` INT, + |`p_cost` DECIMAL(15,2), + |`p_response_target` INT, + |`p_promo_name` CHAR(50), + |`p_channel_dmail` CHAR(1), + |`p_channel_email` CHAR(1), + |`p_channel_catalog` CHAR(1), + |`p_channel_tv` CHAR(1), + |`p_channel_radio` CHAR(1), + |`p_channel_press` CHAR(1), + |`p_channel_event` CHAR(1), + |`p_channel_demo` CHAR(1), + |`p_channel_details` VARCHAR(100), + |`p_purpose` CHAR(15), + |`p_discount_active` CHAR(1) + """.stripMargin, + "reason" -> + """ + |`r_reason_sk` INT, + |`r_reason_id` CHAR(16), + |`r_reason_desc` CHAR(100) + """.stripMargin, + "ship_mode" -> + """ + |`sm_ship_mode_sk` INT, + |`sm_ship_mode_id` CHAR(16), + |`sm_type` CHAR(30), + |`sm_code` CHAR(10), + |`sm_carrier` CHAR(20), + |`sm_contract` CHAR(20) + """.stripMargin, + "time_dim" -> + """ + |`t_time_sk` INT, + |`t_time_id` CHAR(16), + |`t_time` INT, + |`t_hour` INT, + |`t_minute` INT, + |`t_second` INT, + |`t_am_pm` CHAR(2), + |`t_shift` CHAR(20), + |`t_sub_shift` CHAR(20), + |`t_meal_time` CHAR(20) + """.stripMargin + ) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala index 28af552fe586b..13221c9c79c86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UpdateFieldsBenchmark.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.types.{IntegerType, StructField, StructType} * {{{ * To run this benchmark: * 1. without sbt: - * bin/spark-submit --class + * bin/spark-submit --class + * --jars , * 2. with sbt: * build/sbt "sql/test:runMain " * 3. generate result: 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 0e12eba84eb03..eccd6c6b1fff2 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.{BooleanType, LongType, MetadataBuilder, StringType, StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.storage.StorageLevel import org.apache.spark.unsafe.types.UTF8String @@ -1281,8 +1281,8 @@ class DataSourceV2SQLSuite val descriptionDf = sql("DESCRIBE NAMESPACE testcat.ns1.ns2") assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( - ("name", StringType), - ("value", StringType) + ("info_name", StringType), + ("info_value", StringType) )) val description = descriptionDf.collect() assert(description === Seq( @@ -2649,6 +2649,37 @@ class DataSourceV2SQLSuite } } + test("SPARK-34547: metadata columns are resolved last") { + val t1 = s"${catalogAndNamespace}tableOne" + val t2 = "t2" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + withTempView(t2) { + sql(s"CREATE TEMPORARY VIEW $t2 AS SELECT * FROM " + + s"VALUES (1, -1), (2, -2), (3, -3) AS $t2(id, index)") + + val sqlQuery = spark.sql(s"SELECT $t1.id, $t2.id, data, index, $t1.index, $t2.index FROM " + + s"$t1 JOIN $t2 WHERE $t1.id = $t2.id") + val t1Table = spark.table(t1) + val t2Table = spark.table(t2) + val dfQuery = t1Table.join(t2Table, t1Table.col("id") === t2Table.col("id")) + .select(s"$t1.id", s"$t2.id", "data", "index", s"$t1.index", s"$t2.index") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, + Seq( + Row(1, 1, "a", -1, 0, -1), + Row(2, 2, "b", -2, 0, -2), + Row(3, 3, "c", -3, 0, -3) + ) + ) + } + } + } + } + test("SPARK-33505: insert into partitioned table") { val t = "testpart.ns1.ns2.tbl" withTable(t) { @@ -2668,6 +2699,51 @@ class DataSourceV2SQLSuite } } + test("SPARK-34555: Resolve DataFrame metadata column") { + val tbl = s"${catalogAndNamespace}table" + withTable(tbl) { + sql(s"CREATE TABLE $tbl (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $tbl VALUES (1, 'a'), (2, 'b'), (3, 'c')") + val table = spark.table(tbl) + val dfQuery = table.select( + table.col("id"), + table.col("data"), + table.col("index"), + table.col("_partition") + ) + + checkAnswer( + dfQuery, + Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3")) + ) + } + } + + test("SPARK-34561: drop/add columns to a dataset of `DESCRIBE TABLE`") { + val tbl = s"${catalogAndNamespace}tbl" + withTable(tbl) { + sql(s"CREATE TABLE $tbl (c0 INT) USING $v2Format") + val description = sql(s"DESCRIBE TABLE $tbl") + val noCommentDataset = description.drop("comment") + val expectedSchema = new StructType() + .add( + name = "col_name", + dataType = StringType, + nullable = false, + metadata = new MetadataBuilder().putString("comment", "name of the column").build()) + .add( + name = "data_type", + dataType = StringType, + nullable = false, + metadata = new MetadataBuilder().putString("comment", "data type of the column").build()) + assert(noCommentDataset.schema === expectedSchema) + val isNullDataset = noCommentDataset + .withColumn("is_null", noCommentDataset("col_name").isNull) + assert(isNullDataset.schema === expectedSchema.add("is_null", BooleanType, false)) + } + } + private def testNotSupportedV2Command( sqlCommand: String, sqlParams: String, @@ -2686,6 +2762,25 @@ class DataSourceV2SQLSuite assert(e.message.contains(s"$sqlCommand is only supported with v1 tables")) } + test("SPARK-34577: drop/add columns to a dataset of `DESCRIBE NAMESPACE`") { + withNamespace("ns") { + sql("CREATE NAMESPACE ns") + val description = sql(s"DESCRIBE NAMESPACE ns") + val noCommentDataset = description.drop("info_name") + val expectedSchema = new StructType() + .add( + name = "info_value", + dataType = StringType, + nullable = true, + metadata = new MetadataBuilder() + .putString("comment", "value of the namespace info").build()) + assert(noCommentDataset.schema === expectedSchema) + val isNullDataset = noCommentDataset + .withColumn("is_null", noCommentDataset("info_value").isNull) + assert(isNullDataset.schema === expectedSchema.add("is_null", BooleanType, false)) + } + } + private def testV1CommandSupportingTempView(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams") @@ -2699,6 +2794,109 @@ class DataSourceV2SQLSuite }.getMessage assert(errMsg.contains(expectedError)) } + + test("SPARK-34923: do not propagate metadata columns through Project") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + assertThrows[AnalysisException] { + sql(s"SELECT index, _partition from (SELECT id, data FROM $t1)") + } + assertThrows[AnalysisException] { + spark.table(t1).select("id", "data").select("index", "_partition") + } + } + } + + test("SPARK-34923: do not propagate metadata columns through View") { + val t1 = s"${catalogAndNamespace}table" + val view = "view" + + withTable(t1) { + withTempView(view) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + sql(s"CACHE TABLE $view AS SELECT * FROM $t1") + assertThrows[AnalysisException] { + sql(s"SELECT index, _partition FROM $view") + } + } + } + } + + test("SPARK-34923: propagate metadata columns through Filter") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql(s"SELECT id, data, index, _partition FROM $t1 WHERE id > 1") + val dfQuery = spark.table(t1).where("id > 1").select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through Sort") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql(s"SELECT id, data, index, _partition FROM $t1 ORDER BY id") + val dfQuery = spark.table(t1).orderBy("id").select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through RepartitionBy") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql( + s"SELECT /*+ REPARTITION_BY_RANGE(3, id) */ id, data, index, _partition FROM $t1") + val tbl = spark.table(t1) + val dfQuery = tbl.repartitionByRange(3, tbl.col("id")) + .select("id", "data", "index", "_partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } + + test("SPARK-34923: propagate metadata columns through SubqueryAlias") { + val t1 = s"${catalogAndNamespace}table" + val sbq = "sbq" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format " + + "PARTITIONED BY (bucket(4, id), id)") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b'), (3, 'c')") + + val sqlQuery = spark.sql( + s"SELECT $sbq.id, $sbq.data, $sbq.index, $sbq._partition FROM $t1 as $sbq") + val dfQuery = spark.table(t1).as(sbq).select( + s"$sbq.id", s"$sbq.data", s"$sbq.index", s"$sbq._partition") + + Seq(sqlQuery, dfQuery).foreach { query => + checkAnswer(query, Seq(Row(1, "a", 0, "3/1"), Row(2, "b", 0, "0/2"), Row(3, "c", 0, "1/3"))) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala index 2cc7a1f994645..ad730376b2e3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/InsertIntoTests.scala @@ -477,5 +477,15 @@ trait InsertIntoSQLOnlyTests verifyTable(t1, spark.table(view)) } } + + test("SPARK-34599: InsertInto: overwrite - dot in the partition column name - static mode") { + import testImplicits._ + val t1 = "tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 (`a.b` string, `c.d` string) USING $v2Format PARTITIONED BY (`a.b`)") + sql(s"INSERT OVERWRITE $t1 PARTITION (`a.b` = 'a') (`c.d`) VALUES('b')") + verifyTable(t1, Seq("a" -> "b").toDF("id", "data")) + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala index bad21aac41712..ce94d3b5c2fc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TableCapabilityCheckSuite.scala @@ -22,7 +22,7 @@ import java.util import scala.collection.JavaConverters._ import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.analysis.{AnalysisSuite, NamedRelation} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, NamedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2 @@ -36,7 +36,7 @@ import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap -class TableCapabilityCheckSuite extends AnalysisSuite with SharedSparkSession { +class TableCapabilityCheckSuite extends AnalysisTest with SharedSparkSession { private val emptyMap = CaseInsensitiveStringMap.empty private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala index 22c5b651f7e12..4cb8cf1cab9f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/CoalesceShufflePartitionsSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ import org.apache.spark.sql.execution.adaptive._ @@ -57,15 +58,18 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl def withSparkSession( f: SparkSession => Unit, targetPostShuffleInputSize: Int, - minNumPostShufflePartitions: Option[Int]): Unit = { + minNumPostShufflePartitions: Option[Int], + enableIOEncryption: Boolean = false): Unit = { val sparkConf = new SparkConf(false) .setMaster("local[*]") .setAppName("test") .set(UI_ENABLED, false) + .set(IO_ENCRYPTION_ENABLED, enableIOEncryption) .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") .set(SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") + .set(SQLConf.FETCH_SHUFFLE_BLOCKS_IN_BATCH.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES.key, @@ -408,6 +412,25 @@ class CoalesceShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAl } withSparkSession(test, 100, None) } + + test("SPARK-34790: enable IO encryption in AQE partition coalescing") { + val test: SparkSession => Unit = { spark: SparkSession => + val ds = spark.range(0, 100, 1, numInputPartitions) + val resultDf = ds.repartition(ds.col("id")) + resultDf.collect() + + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert( + finalPlan.collect { + case r @ CoalescedShuffleReader() => r + }.isDefinedAt(0)) + } + Seq(true, false).foreach { enableIOEncryption => + // Before SPARK-34790, it will throw an exception when io encryption enabled. + withSparkSession(test, Int.MaxValue, None, enableIOEncryption) + } + } } object CoalescedShuffleReader { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index 0869e25674e69..8962e923cccfb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -47,6 +47,9 @@ object ExternalAppendOnlyUnsafeRowArrayBenchmark extends BenchmarkBase { // for a bug we had with bytes written past the last object in a batch (SPARK-2792) .set(config.SERIALIZER_OBJECT_STREAM_RESET, 1) .set(config.SERIALIZER, "org.apache.spark.serializer.JavaSerializer") + // SPARK-34832: Add this configuration to allow `withFakeTaskContext` method + // to create `SparkContext` on the executor side. + .set(config.EXECUTOR_ALLOW_SPARK_CONTEXT, true) private def withFakeTaskContext(f: => Unit): Unit = { val sc = new SparkContext("local", "test", conf) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala index 68e1a682562ac..c9277802d8563 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.logical.Repartition import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} @@ -258,6 +259,107 @@ abstract class SQLViewTestSuite extends QueryTest with SQLTestUtils { checkViewOutput(viewName, Seq(Row(2))) } } + + test("SPARK-34490 - query should fail if the view refers a dropped table") { + withTable("t") { + Seq(2, 3, 1).toDF("c1").write.format("parquet").saveAsTable("t") + val viewName = createView("testView", "SELECT * FROM t") + withView(viewName) { + // Always create a temp view in this case, not use `createView` on purpose + sql("CREATE TEMP VIEW t AS SELECT 1 AS c1") + withTempView("t") { + checkViewOutput(viewName, Seq(Row(2), Row(3), Row(1))) + // Manually drop table `t` to see if the query will fail + sql("DROP TABLE IF EXISTS default.t") + val e = intercept[AnalysisException] { + sql(s"SELECT * FROM $viewName").collect() + }.getMessage + assert(e.contains("Table or view not found: t")) + } + } + } + } + + test("SPARK-34613: Fix view does not capture disable hint config") { + withSQLConf(DISABLE_HINTS.key -> "true") { + val viewName = createView("v1", "SELECT /*+ repartition(1) */ 1") + withView(viewName) { + assert( + sql(s"SELECT * FROM $viewName").queryExecution.analyzed.collect { + case e: Repartition => e + }.isEmpty + ) + checkViewOutput(viewName, Seq(Row(1))) + } + } + } + + test("SPARK-34504: drop an invalid view") { + withTable("t") { + sql("CREATE TABLE t(s STRUCT) USING json") + val viewName = createView("v", "SELECT s.i FROM t") + withView(viewName) { + assert(spark.table(viewName).collect().isEmpty) + + // re-create the table without nested field `i` which is referred by the view. + sql("DROP TABLE t") + sql("CREATE TABLE t(s STRUCT) USING json") + val e = intercept[AnalysisException](spark.table(viewName)) + assert(e.message.contains("No such struct field i in j")) + + // drop invalid view should be fine + sql(s"DROP VIEW $viewName") + } + } + } + + test("SPARK-34719: view query with duplicated output column names") { + Seq(true, false).foreach { caseSensitive => + withSQLConf(CASE_SENSITIVE.key -> caseSensitive.toString) { + withView("v1", "v2") { + sql("CREATE VIEW v1 AS SELECT 1 a, 2 b") + sql("CREATE VIEW v2 AS SELECT 1 col") + + val viewName = createView( + viewName = "testView", + sqlText = "SELECT *, 1 col, 2 col FROM v1", + columnNames = Seq("c1", "c2", "c3", "c4")) + withView(viewName) { + checkViewOutput(viewName, Seq(Row(1, 2, 1, 2))) + + // One more duplicated column `COL` if caseSensitive=false. + sql("CREATE OR REPLACE VIEW v1 AS SELECT 1 a, 2 b, 3 COL") + if (caseSensitive) { + checkViewOutput(viewName, Seq(Row(1, 2, 1, 2))) + } else { + val e = intercept[AnalysisException](spark.table(viewName).collect()) + assert(e.message.contains("incompatible schema change")) + } + } + + // v1 has 3 columns [a, b, COL], v2 has one column [col], so `testView2` has duplicated + // output column names if caseSensitive=false. + val viewName2 = createView( + viewName = "testView2", + sqlText = "SELECT * FROM v1, v2", + columnNames = Seq("c1", "c2", "c3", "c4")) + withView(viewName2) { + checkViewOutput(viewName2, Seq(Row(1, 2, 3, 1))) + + // One less duplicated column if caseSensitive=false. + sql("CREATE OR REPLACE VIEW v1 AS SELECT 1 a, 2 b") + if (caseSensitive) { + val e = intercept[AnalysisException](spark.table(viewName2).collect()) + assert(e.message.contains("'COL' is not found in '(a,b,col)'")) + } else { + val e = intercept[AnalysisException](spark.table(viewName2).collect()) + assert(e.message.contains("incompatible schema change")) + } + } + } + } + } + } } class LocalTempViewTestSuite extends SQLViewTestSuite with SharedSparkSession { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 92f7f40b54770..f7570c0be728a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.URI import org.apache.log4j.Level +import org.scalatest.PrivateMethodTester import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent, SparkListenerJobStart} import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession, Strategy} @@ -45,7 +46,8 @@ import org.apache.spark.util.Utils class AdaptiveQueryExecSuite extends QueryTest with SharedSparkSession - with AdaptiveSparkPlanHelper { + with AdaptiveSparkPlanHelper + with PrivateMethodTester { import testImplicits._ @@ -869,6 +871,25 @@ class AdaptiveQueryExecSuite } } + test("SPARK-34682: CustomShuffleReaderExec operating on canonicalized plan") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + val (_, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT key FROM testData GROUP BY key") + val readers = collect(adaptivePlan) { + case r: CustomShuffleReaderExec => r + } + assert(readers.length == 1) + val reader = readers.head + val c = reader.canonicalized.asInstanceOf[CustomShuffleReaderExec] + // we can't just call execute() because that has separate checks for canonicalized plans + val ex = intercept[IllegalStateException] { + val doExecute = PrivateMethod[Unit](Symbol("doExecute")) + c.invokePrivate(doExecute()) + } + assert(ex.getMessage === "operating on canonicalized plan") + } + } + test("metrics of the shuffle reader") { withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { val (_, adaptivePlan) = runAdaptiveAndVerifyResult( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 965d78227c335..d77ef6e6bdaf0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -36,7 +36,8 @@ import org.apache.spark.unsafe.map.BytesToBytesMap * Benchmark to measure performance for aggregate primitives. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/AggregateBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala index ae241b3625d02..f78ccf9569a0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala @@ -29,7 +29,8 @@ import org.apache.spark.benchmark.Benchmark * * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/BloomFilterBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala index 6925bdd72674f..361deb0d3e3b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -31,7 +31,8 @@ import org.apache.spark.sql.internal.SQLConf * * To measure specified formats, run it with arguments. * 1. without sbt: - * bin/spark-submit --class format1 [format2] [...] + * bin/spark-submit --class --jars , + * format1 [format2] [...] * 2. build/sbt "sql/test:runMain format1 [format2] [...]" * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt * "sql/test:runMain format1 [format2] [...]" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CharVarcharBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CharVarcharBenchmark.scala index 60f978cb5e4f4..fc867b07c0ff7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CharVarcharBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/CharVarcharBenchmark.scala @@ -25,7 +25,8 @@ import org.apache.spark.benchmark.Benchmark * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index b06ca71b04ecc..4e42330088da5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -32,7 +32,8 @@ import org.apache.spark.sql.internal.SQLConf * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala index bc94d1f235800..3b41226687dc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala @@ -37,7 +37,8 @@ object DateTime extends Enumeration { * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index 8372698fb47ba..b5aa032ce150f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.internal.SQLConf * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index a98ca7f5d8f88..2bd03b6cb758b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -33,7 +33,8 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType} * Benchmark to measure read performance with Filter pushdown. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/FilterPushdownBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala index f3647b3bb2631..f03c22aecfce3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.benchmark -import org.scalatest.Assertions._ - import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED @@ -32,7 +30,8 @@ import org.apache.spark.sql.types.LongType * Benchmark to measure metrics performance at HashedRelation. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/HashedRelationMetricsBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala index 704227e4b4db4..f5a3483bdf3ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -31,7 +31,9 @@ import org.apache.spark.sql.types._ * * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/InExpressionBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertTableWithDynamicPartitionsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertTableWithDynamicPartitionsBenchmark.scala index 81a29cefd0045..32bc2001dcd08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertTableWithDynamicPartitionsBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertTableWithDynamicPartitionsBenchmark.scala @@ -23,7 +23,9 @@ import org.apache.spark.benchmark.Benchmark * Benchmark to measure insert into table with dynamic partition columns. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , < spark sql test jar> * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala index 1cc92892fe122..c20a26e650c6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.benchmark -import org.scalatest.Assertions._ - import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -29,7 +27,8 @@ import org.apache.spark.sql.types.IntegerType * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala index c92098c93aa1e..6ab4fcc9c00fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MakeDateTimeBenchmark.scala @@ -25,7 +25,8 @@ import org.apache.spark.sql.internal.SQLConf * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala index 2aecf553d75a2..18605468adba7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -24,7 +24,9 @@ import org.apache.spark.sql.internal.SQLConf * Benchmark to measure whole stage codegen performance. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/MiscBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala index eeb2605c47c20..a59da45f1fe17 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.internal.SQLConf * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala index e735d1cf257d0..d0289df380d98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala @@ -22,7 +22,8 @@ package org.apache.spark.sql.execution.benchmark * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedPredicatePushDownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedPredicatePushDownBenchmark.scala index f89fe2e64c778..5604f6856b042 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedPredicatePushDownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedPredicatePushDownBenchmark.scala @@ -26,7 +26,8 @@ import org.apache.spark.sql.internal.SQLConf * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala index 1c9cc2c371176..f8f0ab10b24ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala @@ -22,7 +22,8 @@ package org.apache.spark.sql.execution.benchmark * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala index e07921bf3aa74..b09549be167f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala @@ -23,7 +23,9 @@ import org.apache.spark.sql.SparkSession /** * Benchmark primitive arrays via DataFrame and Dataset program using primitive arrays * To run this benchmark: - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , < spark sql test jar> * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/PrimitiveArrayBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala index e566f5d5adee6..e9bdff5853a51 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala @@ -24,7 +24,8 @@ import org.apache.spark.benchmark.Benchmark * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/RangeBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 2c9e8a909633c..8b8710d96b5ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -30,7 +30,8 @@ import org.apache.spark.util.random.XORShiftRandom * Benchmark to measure performance for aggregate primitives. * {{{ * To run this benchmark: - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala index ee8a6e787c36c..34e51efc1f796 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala @@ -28,7 +28,8 @@ import org.apache.spark.sql.types.{IntegerType, StringType} * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 9b0389c6d1ea4..67eb20c42a9ef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData * Benchmark [[UnsafeArrayDataBenchmark]] for UnsafeArrayData * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/UnsafeArrayDataBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index 77dc3a10f8033..c209a6da3a3d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.benchmark import java.io.File -import org.scalatest.Assertions._ - import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame import org.apache.spark.util.Utils @@ -30,7 +28,8 @@ import org.apache.spark.util.Utils * To run this benchmark: * {{{ * 1. without sbt: - * bin/spark-submit --class --jars + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/WideSchemaBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index fcb18392235c3..cad7f7a11211b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -22,7 +22,6 @@ import java.nio.charset.StandardCharsets import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.math3.distribution.LogNormalDistribution -import org.scalatest.Assertions._ import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.catalyst.expressions.GenericInternalRow @@ -34,7 +33,9 @@ import org.apache.spark.util.Utils._ * Benchmark to decoders using various compression schemes. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars , * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/CompressionSchemeBenchmark-results.txt". 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 5c08b066019be..55394ae5cd561 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 @@ -1478,6 +1478,17 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { // set table partition location sql("ALTER TABLE dbx.tab1 PARTITION (a='1', b='2') SET LOCATION '/path/to/part/ways'") verifyLocation(new URI("/path/to/part/ways"), Some(partSpec)) + // set location for partition spec in the upper case + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + sql("ALTER TABLE dbx.tab1 PARTITION (A='1', B='2') SET LOCATION '/path/to/part/ways2'") + verifyLocation(new URI("/path/to/part/ways2"), Some(partSpec)) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val errMsg = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (A='1', B='2') SET LOCATION '/path/to/part/ways3'") + }.getMessage + assert(errMsg.contains("not a valid partition column")) + } // set table location without explicitly specifying database catalog.setCurrentDatabase("dbx") sql("ALTER TABLE tab1 SET LOCATION '/swanky/steak/place'") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala index a34350e4a9676..b3bb098ea5b01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -856,13 +856,13 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed2, expected2) } else { parsed1 match { - case DescribeRelation(_: ResolvedTable, _, isExtended) => + case DescribeRelation(_: ResolvedTable, _, isExtended, _) => assert(!isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed1.treeString) } parsed2 match { - case DescribeRelation(_: ResolvedTable, _, isExtended) => + case DescribeRelation(_: ResolvedTable, _, isExtended, _) => assert(isExtended) case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) } @@ -876,7 +876,7 @@ class PlanResolutionSuite extends AnalysisTest { comparePlans(parsed3, expected3) } else { parsed3 match { - case DescribeRelation(_: ResolvedTable, partitionSpec, isExtended) => + case DescribeRelation(_: ResolvedTable, partitionSpec, isExtended, _) => assert(!isExtended) assert(partitionSpec == Map("a" -> "1")) case _ => fail("Expect DescribeTable, but got:\n" + parsed2.treeString) @@ -1179,7 +1179,7 @@ class PlanResolutionSuite extends AnalysisTest { case AppendData(r: DataSourceV2Relation, _, _, _) => assert(r.catalog.exists(_ == catalogIdent)) assert(r.identifier.exists(_.name() == tableIdent)) - case DescribeRelation(r: ResolvedTable, _, _) => + case DescribeRelation(r: ResolvedTable, _, _, _) => assert(r.catalog == catalogIdent) assert(r.identifier.name() == tableIdent) case ShowTableProperties(r: ResolvedTable, _) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala index 32941d8d2cd11..5756bed6e9da9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/BasicWriteTaskStatsTrackerSuite.scala @@ -148,7 +148,7 @@ class BasicWriteTaskStatsTrackerSuite extends SparkFunSuite { test("Three files, last one empty") { val file1 = new Path(tempDirPath, "f-3-1") val file2 = new Path(tempDirPath, "f-3-2") - val file3 = new Path(tempDirPath, "f-3-2") + val file3 = new Path(tempDirPath, "f-3-3") val tracker = new BasicWriteTaskStatsTracker(conf) tracker.newFile(file1.toString) write1(file1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala index c90732183cb7a..765d2fc584a7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -774,4 +774,46 @@ abstract class SchemaPruningSuite assert(scanSchema === expectedScanSchema) } } + + testSchemaPruning("SPARK-34963: extract case-insensitive struct field from array") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val query1 = spark.table("contacts") + .select("friends.First", "friends.MiDDle") + checkScan(query1, "struct>>") + checkAnswer(query1, + Row(Array.empty[String], Array.empty[String]) :: + Row(Array("Susan"), Array("Z.")) :: + Row(null, null) :: + Row(null, null) :: Nil) + + val query2 = spark.table("contacts") + .where("friends.First is not null") + .select("friends.First", "friends.MiDDle") + checkScan(query2, "struct>>") + checkAnswer(query2, + Row(Array.empty[String], Array.empty[String]) :: + Row(Array("Susan"), Array("Z.")) :: Nil) + } + } + + testSchemaPruning("SPARK-34963: extract case-insensitive struct field from struct") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val query1 = spark.table("contacts") + .select("Name.First", "NAME.MiDDle") + checkScan(query1, "struct>") + checkAnswer(query1, + Row("Jane", "X.") :: + Row("Janet", null) :: + Row("Jim", null) :: + Row("John", "Y.") :: Nil) + + val query2 = spark.table("contacts") + .where("Name.MIDDLE is not null") + .select("Name.First", "NAME.MiDDle") + checkScan(query2, "struct>") + checkAnswer(query2, + Row("Jane", "X.") :: + Row("John", "Y.") :: Nil) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 30f0e45d04eab..3fe6ce773c012 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2452,6 +2452,17 @@ abstract class CSVSuite assert(result.sameElements(exceptResults)) } } + + test("SPARK-34768: counting a long record with ignoreTrailingWhiteSpace set to true") { + val bufSize = 128 + val line = "X" * (bufSize - 1) + "| |" + withTempPath { path => + Seq(line).toDF.write.text(path.getAbsolutePath) + assert(spark.read.format("csv") + .option("delimiter", "|") + .option("ignoreTrailingWhiteSpace", "true").load(path.getAbsolutePath).count() == 1) + } + } } class CSVv1Suite extends CSVSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala index 0e9498b2681e2..71b0325f93732 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuite.scala @@ -57,17 +57,23 @@ class ConnectionProviderSuite extends ConnectionProviderSuiteBase with SharedSpa val db2AppEntry = db2Provider.appEntry(db2Driver, db2Options) // Make sure no authentication for the databases are set - val oldConfig = Configuration.getConfiguration - assert(oldConfig.getAppConfigurationEntry(postgresAppEntry) == null) - assert(oldConfig.getAppConfigurationEntry(db2AppEntry) == null) + val rootConfig = Configuration.getConfiguration + assert(rootConfig.getAppConfigurationEntry(postgresAppEntry) == null) + assert(rootConfig.getAppConfigurationEntry(db2AppEntry) == null) - postgresProvider.setAuthenticationConfigIfNeeded(postgresDriver, postgresOptions) - db2Provider.setAuthenticationConfigIfNeeded(db2Driver, db2Options) + postgresProvider.setAuthenticationConfig(postgresDriver, postgresOptions) + val postgresConfig = Configuration.getConfiguration + + db2Provider.setAuthenticationConfig(db2Driver, db2Options) + val db2Config = Configuration.getConfiguration // Make sure authentication for the databases are set - val newConfig = Configuration.getConfiguration - assert(oldConfig != newConfig) - assert(newConfig.getAppConfigurationEntry(postgresAppEntry) != null) - assert(newConfig.getAppConfigurationEntry(db2AppEntry) != null) + assert(rootConfig != postgresConfig) + assert(rootConfig != db2Config) + // The topmost config in the chain is linked with all the subsequent entries + assert(db2Config.getAppConfigurationEntry(postgresAppEntry) != null) + assert(db2Config.getAppConfigurationEntry(db2AppEntry) != null) + + Configuration.setConfiguration(null) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala index a299841b3c149..f42b17abf31bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProviderSuiteBase.scala @@ -59,16 +59,12 @@ abstract class ConnectionProviderSuiteBase extends SparkFunSuite with BeforeAndA // Make sure no authentication for the database is set assert(Configuration.getConfiguration.getAppConfigurationEntry(providerAppEntry) == null) - // Make sure the first call sets authentication properly + // Make sure setAuthenticationConfig call sets authentication properly val savedConfig = Configuration.getConfiguration - provider.setAuthenticationConfigIfNeeded(driver, options) + provider.setAuthenticationConfig(driver, options) val config = Configuration.getConfiguration assert(savedConfig != config) val appEntry = config.getAppConfigurationEntry(providerAppEntry) assert(appEntry != null) - - // Make sure a second call is not modifying the existing authentication - provider.setAuthenticationConfigIfNeeded(driver, options) - assert(config.getAppConfigurationEntry(providerAppEntry) === appEntry) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala index 5885af82532d4..895b3d85d960b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/DB2ConnectionProviderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class DB2ConnectionProviderSuite extends ConnectionProviderSuiteBase { - test("setAuthenticationConfigIfNeeded must set authentication if not set") { + test("setAuthenticationConfig must set authentication all the time") { val provider = new DB2ConnectionProvider() val driver = registerDriver(provider.driverClass) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala index a5704e842e018..a0b9af2d82e13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala @@ -22,7 +22,7 @@ import java.sql.Driver import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { - test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") { + test("setAuthenticationConfig default parser must set authentication all the time") { val provider = new MSSQLConnectionProvider() val driver = registerDriver(provider.driverClass) @@ -30,7 +30,7 @@ class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) } - test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") { + test("setAuthenticationConfig custom parser must set authentication all the time") { val provider = new MSSQLConnectionProvider() { override val parserMethod: String = "IntentionallyNotExistingMethod" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala index f450662fcbe74..d8bdf26b35c7d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProviderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class MariaDBConnectionProviderSuite extends ConnectionProviderSuiteBase { - test("setAuthenticationConfigIfNeeded must set authentication if not set") { + test("setAuthenticationConfig must set authentication all the time") { val provider = new MariaDBConnectionProvider() val driver = registerDriver(provider.driverClass) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala index 40e7f1191dccc..4aaaf8168eb53 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/OracleConnectionProviderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class OracleConnectionProviderSuite extends ConnectionProviderSuiteBase { - test("setAuthenticationConfigIfNeeded must set authentication if not set") { + test("setAuthenticationConfig must set authentication all the time") { val provider = new OracleConnectionProvider() val driver = registerDriver(provider.driverClass) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala index ee43a7d9708c5..5006bf4091380 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/PostgresConnectionProviderSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.datasources.jdbc.connection class PostgresConnectionProviderSuite extends ConnectionProviderSuiteBase { - test("setAuthenticationConfigIfNeeded must set authentication if not set") { + test("setAuthenticationConfig must set authentication all the time") { val provider = new PostgresConnectionProvider() val defaultOptions = options("jdbc:postgresql://localhost/postgres") val customOptions = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 4c489bdcc649e..40fdcaaf3e6c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -594,4 +594,20 @@ class OrcSourceSuite extends OrcSuite with SharedSparkSession { val df = readResourceOrcFile("test-data/TestStringDictionary.testRowIndex.orc") assert(df.where("str < 'row 001000'").count() === 1000) } + + test("SPARK-34897: Support reconcile schemas based on index after nested column pruning") { + withTable("t1") { + spark.sql( + """ + |CREATE TABLE t1 ( + | _col0 INT, + | _col1 STRING, + | _col2 STRUCT) + |USING ORC + |""".stripMargin) + + spark.sql("INSERT INTO t1 values(1, '2', struct('a', 'b', 'c', 10L))") + checkAnswer(spark.sql("SELECT _col0, _col2.c1 FROM t1"), Seq(Row(1, "a"))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 8f85fe3c52583..9ef43995467c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol +import org.apache.spark.sql.execution.datasources.{SchemaColumnConvertNotSupportedException, SQLHadoopMapReduceCommitProtocol} import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT, SingleElement} import org.apache.spark.sql.execution.datasources.v2.BatchScanExec import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan @@ -840,6 +840,67 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS testMigration(fromTsType = "INT96", toTsType = "TIMESTAMP_MICROS") testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") } + + test("SPARK-34212 Parquet should read decimals correctly") { + def readParquet(schema: String, path: File): DataFrame = { + spark.read.schema(schema).parquet(path.toString) + } + + withTempPath { path => + // a is int-decimal (4 bytes), b is long-decimal (8 bytes), c is binary-decimal (16 bytes) + val df = sql("SELECT 1.0 a, CAST(1.23 AS DECIMAL(17, 2)) b, CAST(1.23 AS DECIMAL(36, 2)) c") + df.write.parquet(path.toString) + + withAllParquetReaders { + // We can read the decimal parquet field with a larger precision, if scale is the same. + val schema = "a DECIMAL(9, 1), b DECIMAL(18, 2), c DECIMAL(38, 2)" + checkAnswer(readParquet(schema, path), df) + } + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + val schema1 = "a DECIMAL(3, 2), b DECIMAL(18, 3), c DECIMAL(37, 3)" + checkAnswer(readParquet(schema1, path), df) + val schema2 = "a DECIMAL(3, 0), b DECIMAL(18, 1), c DECIMAL(37, 1)" + checkAnswer(readParquet(schema2, path), Row(1, 1.2, 1.2)) + } + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + Seq("a DECIMAL(3, 2)", "b DECIMAL(18, 1)", "c DECIMAL(37, 1)").foreach { schema => + val e = intercept[SparkException] { + readParquet(schema, path).collect() + }.getCause.getCause + assert(e.isInstanceOf[SchemaColumnConvertNotSupportedException]) + } + } + } + + // tests for parquet types without decimal metadata. + withTempPath { path => + val df = sql(s"SELECT 1 a, 123456 b, ${Int.MaxValue.toLong * 10} c, CAST('1.2' AS BINARY) d") + df.write.parquet(path.toString) + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + checkAnswer(readParquet("a DECIMAL(3, 2)", path), sql("SELECT 1.00")) + checkAnswer(readParquet("b DECIMAL(3, 2)", path), Row(null)) + checkAnswer(readParquet("b DECIMAL(11, 1)", path), sql("SELECT 123456.0")) + checkAnswer(readParquet("c DECIMAL(11, 1)", path), Row(null)) + checkAnswer(readParquet("c DECIMAL(13, 0)", path), df.select("c")) + val e = intercept[SparkException] { + readParquet("d DECIMAL(3, 2)", path).collect() + }.getCause + assert(e.getMessage.contains("Please read this column/field as Spark BINARY type")) + } + + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + Seq("a DECIMAL(3, 2)", "c DECIMAL(18, 1)", "d DECIMAL(37, 1)").foreach { schema => + val e = intercept[SparkException] { + readParquet(schema, path).collect() + }.getCause.getCause + assert(e.isInstanceOf[SchemaColumnConvertNotSupportedException]) + } + } + } + } } class ParquetV1QuerySuite extends ParquetQuerySuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index fcbc0da9d5551..13848e5cde23b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -103,7 +103,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSparkSession { ProjectExec(output, FilterExec(condition, join)) } - test(s"$testName using ShuffledHashJoin") { + testWithWholeStageCodegenOnAndOff(s"$testName using ShuffledHashJoin") { _ => extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index f476c15f59983..cf05c6be70690 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -153,7 +153,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSparkSession { } } - test(s"$testName using ShuffledHashJoin (build=left)") { + testWithWholeStageCodegenOnAndOff(s"$testName using ShuffledHashJoin (build=left)") { _ => extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => @@ -165,7 +165,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSparkSession { } } - test(s"$testName using ShuffledHashJoin (build=right)") { + testWithWholeStageCodegenOnAndOff(s"$testName using ShuffledHashJoin (build=right)") { _ => extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 9f7e0a14f6a5c..150d40d0301fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -104,18 +104,16 @@ class OuterJoinSuite extends SparkPlanTest with SharedSparkSession { ExtractEquiJoinKeys.unapply(join) } - if (joinType != FullOuter) { - test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => - withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { - val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft - checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => - EnsureRequirements.apply( - ShuffledHashJoinExec( - leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), - expectedAnswer.map(Row.fromTuple), - sortAnswers = true) - } + testWithWholeStageCodegenOnAndOff(s"$testName using ShuffledHashJoin") { _ => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft + checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => + EnsureRequirements.apply( + ShuffledHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, boundCondition, left, right)), + expectedAnswer.map(Row.fromTuple), + sortAnswers = true) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 21d17f40abb34..d5f9875a5926b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.joins.ShuffledHashJoinExec import org.apache.spark.sql.functions._ @@ -755,4 +756,20 @@ class SQLMetricsSuite extends SharedSparkSession with SQLMetricsTestUtils } } } + + test("SPARK-34567: Add metrics for CTAS operator") { + withTable("t") { + val df = sql("CREATE TABLE t USING PARQUET AS SELECT 1 as a") + val dataWritingCommandExec = + df.queryExecution.executedPlan.asInstanceOf[DataWritingCommandExec] + dataWritingCommandExec.executeCollect() + val createTableAsSelect = dataWritingCommandExec.cmd + assert(createTableAsSelect.metrics.contains("numFiles")) + assert(createTableAsSelect.metrics("numFiles").value == 1) + assert(createTableAsSelect.metrics.contains("numOutputBytes")) + assert(createTableAsSelect.metrics("numOutputBytes").value > 0) + assert(createTableAsSelect.metrics.contains("numOutputRows")) + assert(createTableAsSelect.metrics("numOutputRows").value == 1) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala index c09ff51ecaff2..533d98da240fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/MetricsAggregationBenchmark.scala @@ -39,7 +39,8 @@ import org.apache.spark.util.kvstore.InMemoryStore * Benchmark for metrics aggregation in the SQL listener. * {{{ * To run this benchmark: - * 1. without sbt: bin/spark-submit --class --jars + * 1. without sbt: + * bin/spark-submit --class --jars * 2. build/sbt "core/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " * Results will be written to "benchmarks/MetricsAggregationBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala index 953b3a67d976f..f9ae611691a7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -32,7 +32,9 @@ import org.apache.spark.util.collection.BitSet * Benchmark to low level memory access using different ways to manage buffers. * To run this benchmark: * {{{ - * 1. without sbt: bin/spark-submit --class + * 1. without sbt: + * bin/spark-submit --class + * --jars * 2. build/sbt "sql/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " * Results will be written to "benchmarks/ColumnarBatchBenchmark-results.txt". diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 1ea2d4fd0b32c..40570670fb942 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -117,6 +117,21 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } } + test(s"SPARK-35168: ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} should respect" + + s" ${SQLConf.SHUFFLE_PARTITIONS.key}") { + spark.sessionState.conf.clear() + try { + sql(s"SET ${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key}=true") + sql(s"SET ${SQLConf.COALESCE_PARTITIONS_ENABLED.key}=true") + sql(s"SET ${SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key}=1") + sql(s"SET ${SQLConf.SHUFFLE_PARTITIONS.key}=2") + checkAnswer(sql(s"SET ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}"), + Row(SQLConf.SHUFFLE_PARTITIONS.key, "2")) + } finally { + spark.sessionState.conf.clear() + } + } + test("SPARK-31234: reset will not change static sql configs and spark core configs") { val conf = spark.sparkContext.getConf.getAll.toMap val appName = conf.get("spark.app.name") @@ -414,13 +429,12 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "America/Chicago") assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === "America/Chicago") + spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "GMT+8:00") + assert(sql(s"set ${SQLConf.SESSION_LOCAL_TIMEZONE.key}").head().getString(1) === "GMT+8:00") intercept[IllegalArgumentException] { spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "pst") } - intercept[IllegalArgumentException] { - spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "GMT+8:00") - } val e = intercept[IllegalArgumentException] { spark.conf.set(SQLConf.SESSION_LOCAL_TIMEZONE.key, "Asia/shanghai") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index aaf8765c04425..1d0a57a216c32 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import java.io.File +import java.io.{File, IOException} import java.sql.Date import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataOutputStream, Path, RawLocalFileSystem} @@ -797,27 +797,6 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { } } - test("SPARK-30844: static partition should also follow StoreAssignmentPolicy") { - SQLConf.StoreAssignmentPolicy.values.foreach { policy => - withSQLConf( - SQLConf.STORE_ASSIGNMENT_POLICY.key -> policy.toString) { - withTable("t") { - sql("create table t(a int, b string) using parquet partitioned by (a)") - policy match { - case SQLConf.StoreAssignmentPolicy.ANSI | SQLConf.StoreAssignmentPolicy.STRICT => - val errorMsg = intercept[NumberFormatException] { - sql("insert into t partition(a='ansi') values('ansi')") - }.getMessage - assert(errorMsg.contains("invalid input syntax for type numeric: ansi")) - case SQLConf.StoreAssignmentPolicy.LEGACY => - sql("insert into t partition(a='ansi') values('ansi')") - checkAnswer(sql("select * from t"), Row("ansi", null) :: Nil) - } - } - } - } - } - test("SPARK-24860: dynamic partition overwrite specified per source without catalog table") { withTempPath { path => Seq((1, 1), (2, 2)).toDF("i", "part") @@ -954,6 +933,127 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { assert(msg.contains("cannot resolve '`c3`' given input columns")) } } + + test("SPARK-34926: PartitioningUtils.getPathFragment() should respect partition value is null") { + withTable("t1", "t2") { + sql("CREATE TABLE t1(id INT) USING PARQUET") + sql( + """ + |CREATE TABLE t2 (c1 INT, part STRING) + | USING parquet + |PARTITIONED BY (part) + |""".stripMargin) + sql( + """ + |INSERT INTO TABLE t2 PARTITION (part = null) + |SELECT * FROM t1 where 1=0""".stripMargin) + checkAnswer(spark.table("t2"), Nil) + } + } + + test("SPARK-35106: insert overwrite with custom partition path") { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + sql(s"insert into t partition(part1=1, part2=1) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1)) + + sql("insert overwrite table t partition(part1=1, part2=1) select 2") + checkAnswer(spark.table("t"), Row(2, 1, 1)) + + sql("insert overwrite table t partition(part1=2, part2) select 2, 2") + checkAnswer(spark.table("t"), Row(2, 1, 1) :: Row(2, 2, 2) :: Nil) + + sql("insert overwrite table t partition(part1=1, part2=2) select 3") + checkAnswer(spark.table("t"), Row(2, 1, 1) :: Row(2, 2, 2) :: Row(3, 1, 2) :: Nil) + + sql("insert overwrite table t partition(part1=1, part2) select 4, 1") + checkAnswer(spark.table("t"), Row(4, 1, 1) :: Row(2, 2, 2) :: Nil) + } + } + } + + test("SPARK-35106: dynamic partition overwrite with custom partition path") { + withSQLConf(SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"insert into t partition(part1=1, part2=1) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1)) + + sql(s"alter table t add partition(part1=1, part2=2) location '${path.getAbsolutePath}'") + + // dynamic partition overwrite to empty custom partition + sql(s"insert overwrite table t partition(part1=1, part2=2) select 1") + checkAnswer(spark.table("t"), Row(1, 1, 1) :: Row(1, 1, 2) :: Nil) + + // dynamic partition overwrite to non-empty custom partition + sql("insert overwrite table t partition(part1=1, part2=2) select 2") + checkAnswer(spark.table("t"), Row(1, 1, 1) :: Row(2, 1, 2) :: Nil) + } + } + } + } + + test("SPARK-35106: Throw exception when rename custom partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true") { + withTempPath { path => + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + sql(s"alter table t add partition(part1=1, part2=1) location '${path.getAbsolutePath}'") + + val e = intercept[SparkException] { + sql(s"insert into t partition(part1=1, part2=1) select 1") + }.getCause + assert(e.isInstanceOf[IOException]) + assert(e.getMessage.contains("Failed to rename")) + assert(e.getMessage.contains("when committing files staged for absolute location")) + } + } + } + } + + test("SPARK-35106: Throw exception when rename dynamic partition paths returns false") { + withSQLConf( + "fs.file.impl" -> classOf[RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem].getName, + "fs.file.impl.disable.cache" -> "true", + SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString) { + + withTable("t") { + sql( + """ + |create table t(i int, part1 int, part2 int) using parquet + |partitioned by (part1, part2) + """.stripMargin) + + val e = intercept[SparkException] { + sql(s"insert overwrite table t partition(part1, part2) values (1, 1, 1)") + }.getCause + assert(e.isInstanceOf[IOException]) + assert(e.getMessage.contains("Failed to rename")) + assert(e.getMessage.contains( + "when committing files staged for overwriting dynamic partitions")) + } + } + } } class FileExistingTestFileSystem extends RawLocalFileSystem { @@ -966,3 +1066,13 @@ class FileExistingTestFileSystem extends RawLocalFileSystem { throw new FileAlreadyExistsException(s"${f.toString} already exists") } } + +class RenameFromSparkStagingToFinalDirAlwaysTurnsFalseFilesystem extends RawLocalFileSystem { + override def rename(src: Path, dst: Path): Boolean = { + (!isSparkStagingDir(src) || isSparkStagingDir(dst)) && super.rename(src, dst) + } + + private def isSparkStagingDir(path: Path): Boolean = { + path.toString.contains(".spark-staging-") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index ed284df10aced..0d2d00f1f5b2b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -34,7 +34,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkConf, SparkContext, TaskContext, TestUtils} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.catalyst.plans.logical.{Range, RepartitionByExpression} import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan} @@ -1264,6 +1264,37 @@ class StreamSuite extends StreamTest { } } } + + test("SPARK-34482: correct active SparkSession for logicalPlan") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") { + val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() + var query: StreamExecution = null + try { + query = + df.repartition($"a") + .writeStream + .format("memory") + .queryName("memory") + .start() + .asInstanceOf[StreamingQueryWrapper] + .streamingQuery + query.awaitInitialization(streamingTimeout.toMillis) + val plan = query.logicalPlan + val numPartition = plan + .find { _.isInstanceOf[RepartitionByExpression] } + .map(_.asInstanceOf[RepartitionByExpression].numPartitions) + // Before the fix of SPARK-34482, the numPartition is the value of + // `COALESCE_PARTITIONS_INITIAL_PARTITION_NUM`. + assert(numPartition.get === spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS)) + } finally { + if (query != null) { + query.stop() + } + } + } + } } abstract class FakeSource extends StreamSourceProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index 02f91399fce1c..0e2fcfbd46356 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -54,27 +54,31 @@ class ContinuousSuiteBase extends StreamTest { protected def waitForRateSourceCommittedValue( query: ContinuousExecution, - desiredValue: Long, + partitionIdToDesiredValue: Map[Int, Long], maxWaitTimeMs: Long): Unit = { - def readHighestCommittedValue(c: ContinuousExecution): Option[Long] = { + def readCommittedValues(c: ContinuousExecution): Option[Map[Int, Long]] = { c.committedOffsets.lastOption.map { case (_, offset) => offset match { case o: RateStreamOffset => - o.partitionToValueAndRunTimeMs.map { - case (_, ValueRunTimeMsPair(value, _)) => value - }.max + o.partitionToValueAndRunTimeMs.mapValues(_.value).toMap } } } + def reachDesiredValues: Boolean = { + val committedValues = readCommittedValues(query).getOrElse(Map.empty) + partitionIdToDesiredValue.forall { case (key, value) => + committedValues.contains(key) && committedValues(key) > value + } + } + val maxWait = System.currentTimeMillis() + maxWaitTimeMs - while (System.currentTimeMillis() < maxWait && - readHighestCommittedValue(query).getOrElse(Long.MinValue) < desiredValue) { + while (System.currentTimeMillis() < maxWait && !reachDesiredValues) { Thread.sleep(100) } if (System.currentTimeMillis() > maxWait) { logWarning(s"Couldn't reach desired value in $maxWaitTimeMs milliseconds!" + - s"Current highest committed value is ${readHighestCommittedValue(query)}") + s"Current committed values is ${readCommittedValues(query)}") } } @@ -264,7 +268,7 @@ class ContinuousSuite extends ContinuousSuiteBase { val expected = Set(0, 1, 2, 3) val continuousExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.asInstanceOf[ContinuousExecution] - waitForRateSourceCommittedValue(continuousExecution, expected.max, 20 * 1000) + waitForRateSourceCommittedValue(continuousExecution, Map(0 -> 2, 1 -> 3), 20 * 1000) query.stop() val results = spark.read.table("noharness").collect() diff --git a/sql/gen-sql-api-docs.py b/sql/gen-sql-api-docs.py index 2f734093b106c..17631a7352a02 100644 --- a/sql/gen-sql-api-docs.py +++ b/sql/gen-sql-api-docs.py @@ -24,6 +24,106 @@ ExpressionInfo = namedtuple( "ExpressionInfo", "className name usage arguments examples note since deprecated") +_virtual_operator_infos = [ + ExpressionInfo( + className="", + name="!=", + usage="expr1 != expr2 - Returns true if `expr1` is not equal to `expr2`, " + + "or false otherwise.", + arguments="\n Arguments:\n " + + """* expr1, expr2 - the two expressions must be same type or can be casted to + a common type, and must be a type that can be used in equality comparison. + Map type is not supported. For complex types such array/struct, + the data types of fields must be orderable.""", + examples="\n Examples:\n " + + "> SELECT 1 != 2;\n " + + " true\n " + + "> SELECT 1 != '2';\n " + + " true\n " + + "> SELECT true != NULL;\n " + + " NULL\n " + + "> SELECT NULL != NULL;\n " + + " NULL", + note="", + since="1.0.0", + deprecated=""), + ExpressionInfo( + className="", + name="<>", + usage="expr1 != expr2 - Returns true if `expr1` is not equal to `expr2`, " + + "or false otherwise.", + arguments="\n Arguments:\n " + + """* expr1, expr2 - the two expressions must be same type or can be casted to + a common type, and must be a type that can be used in equality comparison. + Map type is not supported. For complex types such array/struct, + the data types of fields must be orderable.""", + examples="\n Examples:\n " + + "> SELECT 1 != 2;\n " + + " true\n " + + "> SELECT 1 != '2';\n " + + " true\n " + + "> SELECT true != NULL;\n " + + " NULL\n " + + "> SELECT NULL != NULL;\n " + + " NULL", + note="", + since="1.0.0", + deprecated=""), + ExpressionInfo( + className="", + name="between", + usage="expr1 [NOT] BETWEEN expr2 AND expr3 - " + + "evaluate if `expr1` is [not] in between `expr2` and `expr3`.", + arguments="", + examples="\n Examples:\n " + + "> SELECT col1 FROM VALUES 1, 3, 5, 7 WHERE col1 BETWEEN 2 AND 5;\n " + + " 3\n " + + " 5", + note="", + since="1.0.0", + deprecated=""), + ExpressionInfo( + className="", + name="case", + usage="CASE expr1 WHEN expr2 THEN expr3 " + + "[WHEN expr4 THEN expr5]* [ELSE expr6] END - " + + "When `expr1` = `expr2`, returns `expr3`; " + + "when `expr1` = `expr4`, return `expr5`; else return `expr6`.", + arguments="\n Arguments:\n " + + "* expr1 - the expression which is one operand of comparison.\n " + + "* expr2, expr4 - the expressions each of which is the other " + + " operand of comparison.\n " + + "* expr3, expr5, expr6 - the branch value expressions and else value expression" + + " should all be same type or coercible to a common type.", + examples="\n Examples:\n " + + "> SELECT CASE col1 WHEN 1 THEN 'one' " + + "WHEN 2 THEN 'two' ELSE '?' END FROM VALUES 1, 2, 3;\n " + + " one\n " + + " two\n " + + " ?\n " + + "> SELECT CASE col1 WHEN 1 THEN 'one' " + + "WHEN 2 THEN 'two' END FROM VALUES 1, 2, 3;\n " + + " one\n " + + " two\n " + + " NULL", + note="", + since="1.0.1", + deprecated=""), + ExpressionInfo( + className="", + name="||", + usage="expr1 || expr2 - Returns the concatenation of `expr1` and `expr2`.", + arguments="", + examples="\n Examples:\n " + + "> SELECT 'Spark' || 'SQL';\n " + + " SparkSQL\n " + + "> SELECT array(1, 2, 3) || array(4, 5) || array(6);\n " + + " [1,2,3,4,5,6]", + note="\n || for arrays is available since 2.4.0.\n", + since="2.3.0", + deprecated="") +] + def _list_function_infos(jvm): """ @@ -32,7 +132,7 @@ def _list_function_infos(jvm): """ jinfos = jvm.org.apache.spark.sql.api.python.PythonSQLUtils.listBuiltinFunctionInfos() - infos = [] + infos = _virtual_operator_infos for jinfo in jinfos: name = jinfo.getName() usage = jinfo.getUsage() diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 99a57f5e4bd83..addb2dca0c463 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml @@ -133,6 +133,10 @@ net.sf.jpam jpam + + commons-cli + commons-cli + target/scala-${scala.binary.version}/classes diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties index a73946c6dc5f3..ad799deea3678 100644 --- a/sql/hive-thriftserver/src/test/resources/log4j.properties +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -24,6 +24,10 @@ log4j.appender.CA.layout=org.apache.log4j.PatternLayout log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n log4j.appender.CA.Threshold = WARN +# SPARK-34128: Suppress undesirable TTransportException warnings, due to THRIFT-4805 +log4j.appender.CA.filter.1=org.apache.log4j.varia.StringMatchFilter +log4j.appender.CA.filter.1.StringToMatch=Thrift error occurred during processing of message +log4j.appender.CA.filter.1.AcceptOnMatch=false #File Appender log4j.appender.FA=org.apache.log4j.FileAppender @@ -32,6 +36,10 @@ log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n +log4j.appender.FA.filter.1=org.apache.log4j.varia.StringMatchFilter +log4j.appender.FA.filter.1.StringToMatch=Thrift error occurred during processing of message +log4j.appender.FA.filter.1.AcceptOnMatch=false + # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = DEBUG diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c1ebca0dab235..b07e34f1cf86e 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../../pom.xml 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 46ebcb72bbec5..019718cc53a95 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 @@ -1081,7 +1081,11 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // scalastyle:on caselocale val actualPartitionPath = new Path(currentFullPath, actualPartitionString) try { - fs.rename(actualPartitionPath, expectedPartitionPath) + fs.mkdirs(expectedPartitionPath) + if(!fs.rename(actualPartitionPath, expectedPartitionPath)) { + throw new IOException(s"Renaming partition path from $actualPartitionPath to " + + s"$expectedPartitionPath returned false") + } } catch { case e: IOException => throw new SparkException("Unable to rename partition path from " + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala index ed088648bc20a..db67480ceb77a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -700,7 +700,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { @@ -715,7 +715,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { } def unapply(values: Set[Any]): Option[Seq[String]] = { - val extractables = values.toSeq.map(valueToLiteralString.lift) + val extractables = values.filter(_ != null).toSeq.map(valueToLiteralString.lift) if (extractables.nonEmpty && extractables.forall(_.isDefined)) { Some(extractables.map(_.get)) } else { @@ -769,7 +769,9 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { case InSet(child, values) if useAdvanced && values.size > inSetThreshold => val dataType = child.dataType - val sortedValues = values.toSeq.sorted(TypeUtils.getInterpretedOrdering(dataType)) + // Skip null here is safe, more details could see at ExtractableLiterals. + val sortedValues = values.filter(_ != null).toSeq + .sorted(TypeUtils.getInterpretedOrdering(dataType)) convert(And(GreaterThanOrEqual(child, Literal(sortedValues.head, dataType)), LessThanOrEqual(child, Literal(sortedValues.last, dataType)))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index ccaa4502d9d2a..283c254b39602 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -55,6 +55,7 @@ trait CreateHiveTableAsSelectBase extends DataWritingCommand { val command = getWritingCommand(catalog, tableDesc, tableExists = true) command.run(sparkSession, child) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -69,6 +70,7 @@ trait CreateHiveTableAsSelectBase extends DataWritingCommand { val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) command.run(sparkSession, child) + DataWritingCommand.propogateMetrics(sparkSession.sparkContext, command, metrics) } catch { case NonFatal(e) => // drop the created table. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala index e71b11e7a3f41..1df91a1174ba4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/InsertIntoHiveTableBenchmark.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.hive.test.TestHive * To run this benchmark: * {{{ * 1. without sbt: bin/spark-submit --class - * --jars ,, + * --jars ,, * * 2. build/sbt "hive/test:runMain " * 3. generate result: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 3f806ad24ca10..46a60efc5b569 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -35,8 +35,7 @@ import org.apache.spark.sql.types.LongType * To run this benchmark: * {{{ * 1. without sbt: bin/spark-submit --class - * --jars ,, - * --packages org.spark-project.hive:hive-exec:1.2.1.spark2 + * --jars ,, * * 2. build/sbt "hive/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index cc278ea7b23c2..5442062d0906d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -191,7 +191,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { // scalastyle:on line.size.limit if (PROCESS_TABLES.testingVersions.isEmpty) { - fail("Fail to get the lates Spark versions to test.") + logError("Fail to get the latest Spark versions to test.") } PROCESS_TABLES.testingVersions.zipWithIndex.foreach { case (version, index) => @@ -229,7 +229,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", unusedJar.toString) - runSparkSubmit(args) + if (PROCESS_TABLES.testingVersions.nonEmpty) runSparkSubmit(args) } } @@ -248,8 +248,8 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils { .map("""""".r.findFirstMatchIn(_).get.group(1)) .filter(_ < org.apache.spark.SPARK_VERSION) } catch { - // do not throw exception during object initialization. - case NonFatal(_) => Seq("3.0.1", "2.4.7") // A temporary fallback to use a specific version + // Do not throw exception during object initialization. + case NonFatal(_) => Nil } versions .filter(v => v.startsWith("3") || !TestUtils.isPythonVersionAtLeast38()) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 77d54ed45a5de..426d93b35063d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.hive.test.{HiveTestJars, TestHiveContext} -import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS +import org.apache.spark.sql.internal.SQLConf.{LEGACY_TIME_PARSER_POLICY, SHUFFLE_PARTITIONS} import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types.{DecimalType, StructType} import org.apache.spark.tags.{ExtendedHiveTest, SlowHiveTest} @@ -337,6 +337,30 @@ class HiveSparkSubmitSuite unusedJar.toString) runSparkSubmit(argsForShowTables) } + + test("SPARK-34772: RebaseDateTime loadRebaseRecords should use Spark classloader " + + "instead of context") { + assume(!SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + + // We need to specify the metastore database location in case of conflict with other hive + // versions. + withTempDir { file => + file.delete() + val metastore = s"jdbc:derby:;databaseName=${file.getAbsolutePath};create=true" + + val args = Seq( + "--class", SPARK_34772.getClass.getName.stripSuffix("$"), + "--name", "SPARK-34772", + "--master", "local-cluster[2,1,1024]", + "--conf", s"${LEGACY_TIME_PARSER_POLICY.key}=LEGACY", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"spark.hadoop.javax.jdo.option.ConnectionURL=$metastore", + unusedJar.toString) + runSparkSubmit(args) + } + } } object SetMetastoreURLTest extends Logging { @@ -845,3 +869,18 @@ object SPARK_18989_DESC_TABLE { } } } + +object SPARK_34772 { + def main(args: Array[String]): Unit = { + val spark = SparkSession.builder() + .config(UI_ENABLED.key, "false") + .enableHiveSupport() + .getOrCreate() + try { + spark.sql("CREATE TABLE t (c int) PARTITIONED BY (p date)") + spark.sql("SELECT * FROM t WHERE p='2021-01-01'").collect() + } finally { + spark.sql("DROP TABLE IF EXISTS t") + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala index 12ed0e5305299..79b34bd141de3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/FiltersSuite.scala @@ -179,5 +179,28 @@ class FiltersSuite extends SparkFunSuite with Logging with PlanTest { } } + test("SPARK-34515: Fix NPE if InSet contains null value during getPartitionsByFilter") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "2") { + val filter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val converted = shim.convertFilters(testTable, Seq(filter), conf.sessionLocalTimeZone) + assert(converted == "(p >= 1 and p <= 2)") + } + } + + test("SPARK-34538: Skip InSet null value during push filter to Hive metastore") { + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val intFilter = InSet(a("p", IntegerType), Set(null, 1, 2)) + val intConverted = shim.convertFilters(testTable, Seq(intFilter), conf.sessionLocalTimeZone) + assert(intConverted == "(p = 1 or p = 2)") + } + + withSQLConf(SQLConf.HIVE_METASTORE_PARTITION_PRUNING_INSET_THRESHOLD.key -> "3") { + val dateFilter = InSet(a("p", DateType), Set(null, + Literal(Date.valueOf("2020-01-01")).eval(), Literal(Date.valueOf("2021-01-01")).eval())) + val dateConverted = shim.convertFilters(testTable, Seq(dateFilter), conf.sessionLocalTimeZone) + assert(dateConverted == "(p = 2020-01-01 or p = 2021-01-01)") + } + } + private def a(name: String, dataType: DataType) = AttributeReference(name, dataType)() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 028be7e7ad279..607daaf26f8a8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -384,11 +384,11 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA val db = "spark_29425_1" sql(s"CREATE DATABASE $db") assert(sql(s"DESCRIBE DATABASE EXTENDED $db") - .where("database_description_item='Owner'") + .where("info_name='Owner'") .collect().head.getString(1) === Utils.getCurrentUserName()) sql(s"ALTER DATABASE $db SET DBPROPERTIES('abc'='xyz')") assert(sql(s"DESCRIBE DATABASE EXTENDED $db") - .where("database_description_item='Owner'") + .where("info_name='Owner'") .collect().head.getString(1) === Utils.getCurrentUserName()) } finally { catalog.reset() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala index 4d6dafd598a2e..a2de43d737704 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLMetricsSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.execution.adaptive.DisableAdaptiveExecutionSuite +import org.apache.spark.sql.execution.command.DataWritingCommandExec import org.apache.spark.sql.execution.metric.SQLMetricsTestUtils +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton // Disable AQE because metric info is different with AQE on/off @@ -34,4 +36,29 @@ class SQLMetricsSuite extends SQLMetricsTestUtils with TestHiveSingleton testMetricsDynamicPartition("hive", "hive", "t1") } } + + test("SPARK-34567: Add metrics for CTAS operator") { + Seq(false, true).foreach { canOptimized => + withSQLConf(HiveUtils.CONVERT_METASTORE_CTAS.key -> canOptimized.toString) { + withTable("t") { + val df = sql(s"CREATE TABLE t STORED AS PARQUET AS SELECT 1 as a") + val dataWritingCommandExec = + df.queryExecution.executedPlan.asInstanceOf[DataWritingCommandExec] + dataWritingCommandExec.executeCollect() + val createTableAsSelect = dataWritingCommandExec.cmd + if (canOptimized) { + assert(createTableAsSelect.isInstanceOf[OptimizedCreateHiveTableAsSelectCommand]) + } else { + assert(createTableAsSelect.isInstanceOf[CreateHiveTableAsSelectCommand]) + } + assert(createTableAsSelect.metrics.contains("numFiles")) + assert(createTableAsSelect.metrics("numFiles").value == 1) + assert(createTableAsSelect.metrics.contains("numOutputBytes")) + assert(createTableAsSelect.metrics("numOutputBytes").value > 0) + assert(createTableAsSelect.metrics.contains("numOutputRows")) + assert(createTableAsSelect.metrics("numOutputRows").value == 1) + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index a26412c5163ec..6acb03e919e68 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -33,8 +33,7 @@ import org.apache.spark.sql.types._ * {{{ * To run this benchmark: * 1. without sbt: bin/spark-submit --class - * --jars ,,,, - * + * --jars ,, * 2. build/sbt "hive/test:runMain " * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " * Results will be written to "benchmarks/OrcReadBenchmark-results.txt". diff --git a/streaming/pom.xml b/streaming/pom.xml index 62db6350c6e8e..e64e269cb39d7 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index f610f0dd86bf3..3b2314dc8d370 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.12 - 3.1.1 + 3.1.2 ../pom.xml